You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by am...@apache.org on 2016/03/15 19:47:58 UTC

[01/23] incubator-beam git commit: [BEAM-11] This is a placeholder to get the TfIdfTest working. Should be replaced by a SparkStateInternals implementation

Repository: incubator-beam
Updated Branches:
  refs/heads/master 46412e5f2 -> a91e11500


[BEAM-11] This is a placeholder to get the TfIdfTest working. Should be replaced by a
SparkStateInternals implementation


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

Branch: refs/heads/master
Commit: dd1c6ccba99a685966a1673d85722e83f5e2e79f
Parents: 7e9aab0
Author: Sela <an...@paypal.com>
Authored: Sat Mar 12 17:28:15 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:38:26 2016 +0200

----------------------------------------------------------------------
 .../org/apache/beam/runners/spark/SparkProcessContext.java   | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd1c6ccb/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
index c634152..f4d2ca0 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
@@ -30,7 +30,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
 import com.google.cloud.dataflow.sdk.util.TimerInternals;
 import com.google.cloud.dataflow.sdk.util.WindowedValue;
 import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import com.google.cloud.dataflow.sdk.util.state.*;
 import com.google.cloud.dataflow.sdk.values.PCollectionView;
 import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.AbstractIterator;
@@ -154,8 +154,10 @@ abstract class SparkProcessContext<I, O, V> extends DoFn<I, O>.ProcessContext {
 
       @Override
       public StateInternals stateInternals() {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#stateInternals() is not yet supported.");
+        //TODO: implement state internals.
+        // This is a temporary placeholder to get the TfIdfTest
+        // working for the initial Beam code drop.
+        return InMemoryStateInternals.forKey("DUMMY");
       }
 
       @Override


[07/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
new file mode 100644
index 0000000..e64f89a
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
@@ -0,0 +1,805 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.avro.mapred.AvroKey;
+import org.apache.avro.mapreduce.AvroJob;
+import org.apache.avro.mapreduce.AvroKeyInputFormat;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
+import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper;
+import org.apache.beam.runners.spark.io.hadoop.TemplatedAvroKeyOutputFormat;
+import org.apache.beam.runners.spark.io.hadoop.TemplatedTextOutputFormat;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.beam.runners.spark.util.ByteArray;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Tuple2;
+
+/**
+ * Supports translation between a DataFlow transform, and Spark's operations on RDDs.
+ */
+public final class TransformTranslator {
+
+  private TransformTranslator() {
+  }
+
+  public static class FieldGetter {
+    private final Map<String, Field> fields;
+
+    public FieldGetter(Class<?> clazz) {
+      this.fields = Maps.newHashMap();
+      for (Field f : clazz.getDeclaredFields()) {
+        f.setAccessible(true);
+        this.fields.put(f.getName(), f);
+      }
+    }
+
+    public <T> T get(String fieldname, Object value) {
+      try {
+        @SuppressWarnings("unchecked")
+        T fieldValue = (T) fields.get(fieldname).get(value);
+        return fieldValue;
+      } catch (IllegalAccessException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+  }
+
+  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
+    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
+        PCollectionList<T> pcs = context.getInput(transform);
+        JavaRDD<WindowedValue<T>>[] rdds = new JavaRDD[pcs.size()];
+        for (int i = 0; i < rdds.length; i++) {
+          rdds[i] = (JavaRDD<WindowedValue<T>>) context.getRDD(pcs.get(i));
+        }
+        JavaRDD<WindowedValue<T>> rdd = context.getSparkContext().union(rdds);
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>> gbk() {
+    return new TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>>() {
+      @Override
+      public void evaluate(GroupByKey.GroupByKeyOnly<K, V> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, V>>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context.getInputRDD(transform);
+        @SuppressWarnings("unchecked")
+        KvCoder<K, V> coder = (KvCoder<K, V>) context.getInput(transform).getCoder();
+        Coder<K> keyCoder = coder.getKeyCoder();
+        Coder<V> valueCoder = coder.getValueCoder();
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaRDDLike<WindowedValue<KV<K, Iterable<V>>>, ?> outRDD = fromPair(
+              toPair(inRDD.map(WindowingHelpers.<KV<K, V>>unwindowFunction()))
+            .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder))
+            .groupByKey()
+            .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder)))
+            // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
+            .map(WindowingHelpers.<KV<K, Iterable<V>>>windowFunction());
+        context.setOutputRDD(transform, outRDD);
+      }
+    };
+  }
+
+  private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class);
+
+  private static <K, VI, VO> TransformEvaluator<Combine.GroupedValues<K, VI, VO>> grouped() {
+    return new TransformEvaluator<Combine.GroupedValues<K, VI, VO>>() {
+      @Override
+      public void evaluate(Combine.GroupedValues<K, VI, VO> transform, EvaluationContext context) {
+        Combine.KeyedCombineFn<K, VI, ?, VO> keyed = GROUPED_FG.get("fn", transform);
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?>) context.getInputRDD(transform);
+        context.setOutputRDD(transform,
+            inRDD.map(new KVFunction<>(keyed)));
+      }
+    };
+  }
+
+  private static final FieldGetter COMBINE_GLOBALLY_FG = new FieldGetter(Combine.Globally.class);
+
+  private static <I, A, O> TransformEvaluator<Combine.Globally<I, O>> combineGlobally() {
+    return new TransformEvaluator<Combine.Globally<I, O>>() {
+
+      @Override
+      public void evaluate(Combine.Globally<I, O> transform, EvaluationContext context) {
+        final Combine.CombineFn<I, A, O> globally = COMBINE_GLOBALLY_FG.get("fn", transform);
+
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRdd =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+
+        final Coder<I> iCoder = context.getInput(transform).getCoder();
+        final Coder<A> aCoder;
+        try {
+          aCoder = globally.getAccumulatorCoder(
+              context.getPipeline().getCoderRegistry(), iCoder);
+        } catch (CannotProvideCoderException e) {
+          throw new IllegalStateException("Could not determine coder for accumulator", e);
+        }
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaRDD<byte[]> inRddBytes = inRdd
+            .map(WindowingHelpers.<I>unwindowFunction())
+            .map(CoderHelpers.toByteFunction(iCoder));
+
+        /*A*/ byte[] acc = inRddBytes.aggregate(
+            CoderHelpers.toByteArray(globally.createAccumulator(), aCoder),
+            new Function2</*A*/ byte[], /*I*/ byte[], /*A*/ byte[]>() {
+              @Override
+              public /*A*/ byte[] call(/*A*/ byte[] ab, /*I*/ byte[] ib) throws Exception {
+                A a = CoderHelpers.fromByteArray(ab, aCoder);
+                I i = CoderHelpers.fromByteArray(ib, iCoder);
+                return CoderHelpers.toByteArray(globally.addInput(a, i), aCoder);
+              }
+            },
+            new Function2</*A*/ byte[], /*A*/ byte[], /*A*/ byte[]>() {
+              @Override
+              public /*A*/ byte[] call(/*A*/ byte[] a1b, /*A*/ byte[] a2b) throws Exception {
+                A a1 = CoderHelpers.fromByteArray(a1b, aCoder);
+                A a2 = CoderHelpers.fromByteArray(a2b, aCoder);
+                // don't use Guava's ImmutableList.of as values may be null
+                List<A> accumulators = Collections.unmodifiableList(Arrays.asList(a1, a2));
+                A merged = globally.mergeAccumulators(accumulators);
+                return CoderHelpers.toByteArray(merged, aCoder);
+              }
+            }
+        );
+        O output = globally.extractOutput(CoderHelpers.fromByteArray(acc, aCoder));
+
+        Coder<O> coder = context.getOutput(transform).getCoder();
+        JavaRDD<byte[]> outRdd = context.getSparkContext().parallelize(
+            // don't use Guava's ImmutableList.of as output may be null
+            CoderHelpers.toByteArrays(Collections.singleton(output), coder));
+        context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder))
+            .map(WindowingHelpers.<O>windowFunction()));
+      }
+    };
+  }
+
+  private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class);
+
+  private static <K, VI, VA, VO> TransformEvaluator<Combine.PerKey<K, VI, VO>> combinePerKey() {
+    return new TransformEvaluator<Combine.PerKey<K, VI, VO>>() {
+      @Override
+      public void evaluate(Combine.PerKey<K, VI, VO> transform, EvaluationContext context) {
+        final Combine.KeyedCombineFn<K, VI, VA, VO> keyed =
+            COMBINE_PERKEY_FG.get("fn", transform);
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, VI>>, ?> inRdd =
+            (JavaRDDLike<WindowedValue<KV<K, VI>>, ?>) context.getInputRDD(transform);
+
+        @SuppressWarnings("unchecked")
+        KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
+        Coder<K> keyCoder = inputCoder.getKeyCoder();
+        Coder<VI> viCoder = inputCoder.getValueCoder();
+        Coder<VA> vaCoder;
+        try {
+          vaCoder = keyed.getAccumulatorCoder(
+              context.getPipeline().getCoderRegistry(), keyCoder, viCoder);
+        } catch (CannotProvideCoderException e) {
+          throw new IllegalStateException("Could not determine coder for accumulator", e);
+        }
+        Coder<KV<K, VI>> kviCoder = KvCoder.of(keyCoder, viCoder);
+        Coder<KV<K, VA>> kvaCoder = KvCoder.of(keyCoder, vaCoder);
+
+        // We need to duplicate K as both the key of the JavaPairRDD as well as inside the value,
+        // since the functions passed to combineByKey don't receive the associated key of each
+        // value, and we need to map back into methods in Combine.KeyedCombineFn, which each
+        // require the key in addition to the VI's and VA's being merged/accumulated. Once Spark
+        // provides a way to include keys in the arguments of combine/merge functions, we won't
+        // need to duplicate the keys anymore.
+
+        // Key has to bw windowed in order to group by window as well
+        JavaPairRDD<WindowedValue<K>, WindowedValue<KV<K, VI>>> inRddDuplicatedKeyPair =
+            inRdd.mapToPair(
+                new PairFunction<WindowedValue<KV<K, VI>>, WindowedValue<K>,
+                    WindowedValue<KV<K, VI>>>() {
+                  @Override
+                  public Tuple2<WindowedValue<K>,
+                      WindowedValue<KV<K, VI>>> call(WindowedValue<KV<K, VI>> kv) {
+                    WindowedValue<K> wk = WindowedValue.of(kv.getValue().getKey(),
+                        kv.getTimestamp(), kv.getWindows(), kv.getPane());
+                    return new Tuple2<>(wk, kv);
+                  }
+                });
+        //-- windowed coders
+        final WindowedValue.FullWindowedValueCoder<K> wkCoder =
+                WindowedValue.FullWindowedValueCoder.of(keyCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+        final WindowedValue.FullWindowedValueCoder<KV<K, VI>> wkviCoder =
+                WindowedValue.FullWindowedValueCoder.of(kviCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+        final WindowedValue.FullWindowedValueCoder<KV<K, VA>> wkvaCoder =
+                WindowedValue.FullWindowedValueCoder.of(kvaCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaPairRDD<ByteArray, byte[]> inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair
+            .mapToPair(CoderHelpers.toByteFunction(wkCoder, wkviCoder));
+
+        // The output of combineByKey will be "VA" (accumulator) types rather than "VO" (final
+        // output types) since Combine.CombineFn only provides ways to merge VAs, and no way
+        // to merge VOs.
+        JavaPairRDD</*K*/ ByteArray, /*KV<K, VA>*/ byte[]> accumulatedBytes =
+            inRddDuplicatedKeyPairBytes.combineByKey(
+            new Function</*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VI>*/ byte[] input) {
+                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
+                VA va = keyed.createAccumulator(wkvi.getValue().getKey());
+                va = keyed.addInput(wkvi.getValue().getKey(), va, wkvi.getValue().getValue());
+                WindowedValue<KV<K, VA>> wkva =
+                    WindowedValue.of(KV.of(wkvi.getValue().getKey(), va), wkvi.getTimestamp(),
+                    wkvi.getWindows(), wkvi.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            },
+            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc,
+                  /*KV<K, VI>*/ byte[] input) {
+                WindowedValue<KV<K, VA>> wkva = CoderHelpers.fromByteArray(acc, wkvaCoder);
+                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
+                VA va = keyed.addInput(wkva.getValue().getKey(), wkva.getValue().getValue(),
+                    wkvi.getValue().getValue());
+                wkva = WindowedValue.of(KV.of(wkva.getValue().getKey(), va), wkva.getTimestamp(),
+                    wkva.getWindows(), wkva.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            },
+            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc1,
+                  /*KV<K, VA>*/ byte[] acc2) {
+                WindowedValue<KV<K, VA>> wkva1 = CoderHelpers.fromByteArray(acc1, wkvaCoder);
+                WindowedValue<KV<K, VA>> wkva2 = CoderHelpers.fromByteArray(acc2, wkvaCoder);
+                VA va = keyed.mergeAccumulators(wkva1.getValue().getKey(),
+                    // don't use Guava's ImmutableList.of as values may be null
+                    Collections.unmodifiableList(Arrays.asList(wkva1.getValue().getValue(),
+                    wkva2.getValue().getValue())));
+                WindowedValue<KV<K, VA>> wkva = WindowedValue.of(KV.of(wkva1.getValue().getKey(),
+                    va), wkva1.getTimestamp(), wkva1.getWindows(), wkva1.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            });
+
+        JavaPairRDD<WindowedValue<K>, WindowedValue<VO>> extracted = accumulatedBytes
+            .mapToPair(CoderHelpers.fromByteFunction(wkCoder, wkvaCoder))
+            .mapValues(
+                new Function<WindowedValue<KV<K, VA>>, WindowedValue<VO>>() {
+                  @Override
+                  public WindowedValue<VO> call(WindowedValue<KV<K, VA>> acc) {
+                    return WindowedValue.of(keyed.extractOutput(acc.getValue().getKey(),
+                        acc.getValue().getValue()), acc.getTimestamp(),
+                        acc.getWindows(), acc.getPane());
+                  }
+                });
+
+        context.setOutputRDD(transform,
+            fromPair(extracted)
+            .map(new Function<KV<WindowedValue<K>, WindowedValue<VO>>, WindowedValue<KV<K, VO>>>() {
+              @Override
+              public WindowedValue<KV<K, VO>> call(KV<WindowedValue<K>, WindowedValue<VO>> kwvo)
+                  throws Exception {
+                WindowedValue<VO> wvo = kwvo.getValue();
+                KV<K, VO> kvo = KV.of(kwvo.getKey().getValue(), wvo.getValue());
+                return WindowedValue.of(kvo, wvo.getTimestamp(), wvo.getWindows(), wvo.getPane());
+              }
+            }));
+      }
+    };
+  }
+
+  private static final class KVFunction<K, VI, VO>
+      implements Function<WindowedValue<KV<K, Iterable<VI>>>, WindowedValue<KV<K, VO>>> {
+    private final Combine.KeyedCombineFn<K, VI, ?, VO> keyed;
+
+     KVFunction(Combine.KeyedCombineFn<K, VI, ?, VO> keyed) {
+      this.keyed = keyed;
+    }
+
+    @Override
+    public WindowedValue<KV<K, VO>> call(WindowedValue<KV<K, Iterable<VI>>> windowedKv)
+        throws Exception {
+      KV<K, Iterable<VI>> kv = windowedKv.getValue();
+      return WindowedValue.of(KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())),
+          windowedKv.getTimestamp(), windowedKv.getWindows(), windowedKv.getPane());
+    }
+  }
+
+  private static <K, V> JavaPairRDD<K, V> toPair(JavaRDDLike<KV<K, V>, ?> rdd) {
+    return rdd.mapToPair(new PairFunction<KV<K, V>, K, V>() {
+      @Override
+      public Tuple2<K, V> call(KV<K, V> kv) {
+        return new Tuple2<>(kv.getKey(), kv.getValue());
+      }
+    });
+  }
+
+  private static <K, V> JavaRDDLike<KV<K, V>, ?> fromPair(JavaPairRDD<K, V> rdd) {
+    return rdd.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+      @Override
+      public KV<K, V> call(Tuple2<K, V> t2) {
+        return KV.of(t2._1(), t2._2());
+      }
+    });
+  }
+
+  private static <I, O> TransformEvaluator<ParDo.Bound<I, O>> parDo() {
+    return new TransformEvaluator<ParDo.Bound<I, O>>() {
+      @Override
+      public void evaluate(ParDo.Bound<I, O> transform, EvaluationContext context) {
+        DoFnFunction<I, O> dofn =
+            new DoFnFunction<>(transform.getFn(),
+                context.getRuntimeContext(),
+                getSideInputs(transform.getSideInputs(), context));
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+        context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
+      }
+    };
+  }
+
+  private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class);
+
+  private static <I, O> TransformEvaluator<ParDo.BoundMulti<I, O>> multiDo() {
+    return new TransformEvaluator<ParDo.BoundMulti<I, O>>() {
+      @Override
+      public void evaluate(ParDo.BoundMulti<I, O> transform, EvaluationContext context) {
+        TupleTag<O> mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform);
+        MultiDoFnFunction<I, O> multifn = new MultiDoFnFunction<>(
+            transform.getFn(),
+            context.getRuntimeContext(),
+            mainOutputTag,
+            getSideInputs(transform.getSideInputs(), context));
+
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+        JavaPairRDD<TupleTag<?>, WindowedValue<?>> all = inRDD
+            .mapPartitionsToPair(multifn)
+            .cache();
+
+        PCollectionTuple pct = context.getOutput(transform);
+        for (Map.Entry<TupleTag<?>, PCollection<?>> e : pct.getAll().entrySet()) {
+          @SuppressWarnings("unchecked")
+          JavaPairRDD<TupleTag<?>, WindowedValue<?>> filtered =
+              all.filter(new TupleTagFilter(e.getKey()));
+          @SuppressWarnings("unchecked")
+          // Object is the best we can do since different outputs can have different tags
+          JavaRDD<WindowedValue<Object>> values =
+              (JavaRDD<WindowedValue<Object>>) (JavaRDD<?>) filtered.values();
+          context.setRDD(e.getValue(), values);
+        }
+      }
+    };
+  }
+
+
+  private static <T> TransformEvaluator<TextIO.Read.Bound<T>> readText() {
+    return new TransformEvaluator<TextIO.Read.Bound<T>>() {
+      @Override
+      public void evaluate(TextIO.Read.Bound<T> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaRDD<WindowedValue<String>> rdd = context.getSparkContext().textFile(pattern)
+                .map(WindowingHelpers.<String>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<TextIO.Write.Bound<T>> writeText() {
+    return new TransformEvaluator<TextIO.Write.Bound<T>>() {
+      @Override
+      public void evaluate(TextIO.Write.Bound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<T, Void> last =
+            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
+            .map(WindowingHelpers.<T>unwindowFunction())
+            .mapToPair(new PairFunction<T, T,
+                    Void>() {
+              @Override
+              public Tuple2<T, Void> call(T t) throws Exception {
+                return new Tuple2<>(t, null);
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+                transform.getShardTemplate(), transform.getFilenamePrefix(),
+                transform.getFilenameSuffix());
+        writeHadoopFile(last, new Configuration(), shardTemplateInfo, Text.class,
+            NullWritable.class, TemplatedTextOutputFormat.class);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<AvroIO.Read.Bound<T>> readAvro() {
+    return new TransformEvaluator<AvroIO.Read.Bound<T>>() {
+      @Override
+      public void evaluate(AvroIO.Read.Bound<T> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaSparkContext jsc = context.getSparkContext();
+        @SuppressWarnings("unchecked")
+        JavaRDD<AvroKey<T>> avroFile = (JavaRDD<AvroKey<T>>) (JavaRDD<?>)
+            jsc.newAPIHadoopFile(pattern,
+                                 AvroKeyInputFormat.class,
+                                 AvroKey.class, NullWritable.class,
+                                 new Configuration()).keys();
+        JavaRDD<WindowedValue<T>> rdd = avroFile.map(
+            new Function<AvroKey<T>, T>() {
+              @Override
+              public T call(AvroKey<T> key) {
+                return key.datum();
+              }
+            }).map(WindowingHelpers.<T>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<AvroIO.Write.Bound<T>> writeAvro() {
+    return new TransformEvaluator<AvroIO.Write.Bound<T>>() {
+      @Override
+      public void evaluate(AvroIO.Write.Bound<T> transform, EvaluationContext context) {
+        Job job;
+        try {
+          job = Job.getInstance();
+        } catch (IOException e) {
+          throw new IllegalStateException(e);
+        }
+        AvroJob.setOutputKeySchema(job, transform.getSchema());
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<AvroKey<T>, NullWritable> last =
+            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
+            .map(WindowingHelpers.<T>unwindowFunction())
+            .mapToPair(new PairFunction<T, AvroKey<T>, NullWritable>() {
+              @Override
+              public Tuple2<AvroKey<T>, NullWritable> call(T t) throws Exception {
+                return new Tuple2<>(new AvroKey<>(t), NullWritable.get());
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+            transform.getShardTemplate(), transform.getFilenamePrefix(),
+            transform.getFilenameSuffix());
+        writeHadoopFile(last, job.getConfiguration(), shardTemplateInfo,
+            AvroKey.class, NullWritable.class, TemplatedAvroKeyOutputFormat.class);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<HadoopIO.Read.Bound<K, V>> readHadoop() {
+    return new TransformEvaluator<HadoopIO.Read.Bound<K, V>>() {
+      @Override
+      public void evaluate(HadoopIO.Read.Bound<K, V> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaSparkContext jsc = context.getSparkContext();
+        @SuppressWarnings ("unchecked")
+        JavaPairRDD<K, V> file = jsc.newAPIHadoopFile(pattern,
+            transform.getFormatClass(),
+            transform.getKeyClass(), transform.getValueClass(),
+            new Configuration());
+        JavaRDD<WindowedValue<KV<K, V>>> rdd =
+            file.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+          @Override
+          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
+            return KV.of(t2._1(), t2._2());
+          }
+        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<HadoopIO.Write.Bound<K, V>> writeHadoop() {
+    return new TransformEvaluator<HadoopIO.Write.Bound<K, V>>() {
+      @Override
+      public void evaluate(HadoopIO.Write.Bound<K, V> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<K, V> last = ((JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context
+            .getInputRDD(transform))
+            .map(WindowingHelpers.<KV<K, V>>unwindowFunction())
+            .mapToPair(new PairFunction<KV<K, V>, K, V>() {
+              @Override
+              public Tuple2<K, V> call(KV<K, V> t) throws Exception {
+                return new Tuple2<>(t.getKey(), t.getValue());
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+                transform.getShardTemplate(), transform.getFilenamePrefix(),
+                transform.getFilenameSuffix());
+        Configuration conf = new Configuration();
+        for (Map.Entry<String, String> e : transform.getConfigurationProperties().entrySet()) {
+          conf.set(e.getKey(), e.getValue());
+        }
+        writeHadoopFile(last, conf, shardTemplateInfo,
+            transform.getKeyClass(), transform.getValueClass(), transform.getFormatClass());
+      }
+    };
+  }
+
+  private static final class ShardTemplateInformation {
+    private final int numShards;
+    private final String shardTemplate;
+    private final String filenamePrefix;
+    private final String filenameSuffix;
+
+    private ShardTemplateInformation(int numShards, String shardTemplate, String
+        filenamePrefix, String filenameSuffix) {
+      this.numShards = numShards;
+      this.shardTemplate = shardTemplate;
+      this.filenamePrefix = filenamePrefix;
+      this.filenameSuffix = filenameSuffix;
+    }
+
+    int getNumShards() {
+      return numShards;
+    }
+
+    String getShardTemplate() {
+      return shardTemplate;
+    }
+
+    String getFilenamePrefix() {
+      return filenamePrefix;
+    }
+
+    String getFilenameSuffix() {
+      return filenameSuffix;
+    }
+  }
+
+  private static <K, V> void writeHadoopFile(JavaPairRDD<K, V> rdd, Configuration conf,
+      ShardTemplateInformation shardTemplateInfo, Class<?> keyClass, Class<?> valueClass,
+      Class<? extends FileOutputFormat> formatClass) {
+    int numShards = shardTemplateInfo.getNumShards();
+    String shardTemplate = shardTemplateInfo.getShardTemplate();
+    String filenamePrefix = shardTemplateInfo.getFilenamePrefix();
+    String filenameSuffix = shardTemplateInfo.getFilenameSuffix();
+    if (numShards != 0) {
+      // number of shards was set explicitly, so repartition
+      rdd = rdd.repartition(numShards);
+    }
+    int actualNumShards = rdd.partitions().size();
+    String template = replaceShardCount(shardTemplate, actualNumShards);
+    String outputDir = getOutputDirectory(filenamePrefix, template);
+    String filePrefix = getOutputFilePrefix(filenamePrefix, template);
+    String fileTemplate = getOutputFileTemplate(filenamePrefix, template);
+
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_PREFIX, filePrefix);
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_TEMPLATE, fileTemplate);
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_SUFFIX, filenameSuffix);
+    rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
+  }
+
+  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
+
+  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
+    return new TransformEvaluator<Window.Bound<T>>() {
+      @Override
+      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<T>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform);
+        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
+        if (windowFn instanceof GlobalWindows) {
+          context.setOutputRDD(transform, inRDD);
+        } else {
+          @SuppressWarnings("unchecked")
+          DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
+          DoFnFunction<T, T> dofn =
+                  new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null);
+          context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
+        }
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Create.Values<T>> create() {
+    return new TransformEvaluator<Create.Values<T>>() {
+      @Override
+      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
+        Iterable<T> elems = transform.getElements();
+        // Use a coder to convert the objects in the PCollection to byte arrays, so they
+        // can be transferred over the network.
+        Coder<T> coder = context.getOutput(transform).getCoder();
+        context.setOutputRDDFromValues(transform, elems, coder);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<View.AsSingleton<T>> viewAsSingleton() {
+    return new TransformEvaluator<View.AsSingleton<T>>() {
+      @Override
+      public void evaluate(View.AsSingleton<T> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+                context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<View.AsIterable<T>> viewAsIter() {
+    return new TransformEvaluator<View.AsIterable<T>>() {
+      @Override
+      public void evaluate(View.AsIterable<T> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+                context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static <R, W> TransformEvaluator<View.CreatePCollectionView<R, W>> createPCollView() {
+    return new TransformEvaluator<View.CreatePCollectionView<R, W>>() {
+      @Override
+      public void evaluate(View.CreatePCollectionView<R, W> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+            context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static final class TupleTagFilter<V>
+      implements Function<Tuple2<TupleTag<V>, WindowedValue<?>>, Boolean> {
+
+    private final TupleTag<V> tag;
+
+    private TupleTagFilter(TupleTag<V> tag) {
+      this.tag = tag;
+    }
+
+    @Override
+    public Boolean call(Tuple2<TupleTag<V>, WindowedValue<?>> input) {
+      return tag.equals(input._1());
+    }
+  }
+
+  private static Map<TupleTag<?>, BroadcastHelper<?>> getSideInputs(
+      List<PCollectionView<?>> views,
+      EvaluationContext context) {
+    if (views == null) {
+      return ImmutableMap.of();
+    } else {
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs = Maps.newHashMap();
+      for (PCollectionView<?> view : views) {
+        Iterable<? extends WindowedValue<?>> collectionView = context.getPCollectionView(view);
+        Coder<Iterable<WindowedValue<?>>> coderInternal = view.getCoderInternal();
+        @SuppressWarnings("unchecked")
+        BroadcastHelper<?> helper =
+            BroadcastHelper.create((Iterable<WindowedValue<?>>) collectionView, coderInternal);
+        //broadcast side inputs
+        helper.broadcast(context.getSparkContext());
+        sideInputs.put(view.getTagInternal(), helper);
+      }
+      return sideInputs;
+    }
+  }
+
+  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
+      .newHashMap();
+
+  static {
+    EVALUATORS.put(TextIO.Read.Bound.class, readText());
+    EVALUATORS.put(TextIO.Write.Bound.class, writeText());
+    EVALUATORS.put(AvroIO.Read.Bound.class, readAvro());
+    EVALUATORS.put(AvroIO.Write.Bound.class, writeAvro());
+    EVALUATORS.put(HadoopIO.Read.Bound.class, readHadoop());
+    EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop());
+    EVALUATORS.put(ParDo.Bound.class, parDo());
+    EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
+    EVALUATORS.put(GroupByKey.GroupByKeyOnly.class, gbk());
+    EVALUATORS.put(Combine.GroupedValues.class, grouped());
+    EVALUATORS.put(Combine.Globally.class, combineGlobally());
+    EVALUATORS.put(Combine.PerKey.class, combinePerKey());
+    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
+    EVALUATORS.put(Create.Values.class, create());
+    EVALUATORS.put(View.AsSingleton.class, viewAsSingleton());
+    EVALUATORS.put(View.AsIterable.class, viewAsIter());
+    EVALUATORS.put(View.CreatePCollectionView.class, createPCollView());
+    EVALUATORS.put(Window.Bound.class, window());
+  }
+
+  public static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
+  getTransformEvaluator(Class<PT> clazz) {
+    @SuppressWarnings("unchecked")
+    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
+    if (transform == null) {
+      throw new IllegalStateException("No TransformEvaluator registered for " + clazz);
+    }
+    return transform;
+  }
+
+  /**
+   * Translator matches Dataflow transformation with the appropriate evaluator.
+   */
+  public static class Translator implements SparkPipelineTranslator {
+
+    @Override
+    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
+      return EVALUATORS.containsKey(clazz);
+    }
+
+    @Override
+    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
+      return getTransformEvaluator(clazz);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
new file mode 100644
index 0000000..6b904f7
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.spark.api.java.function.Function;
+
+/**
+ * Helper functions for working with windows.
+ */
+public final class WindowingHelpers {
+  private WindowingHelpers() {
+  }
+
+  /**
+   * A function for converting a value to a {@link WindowedValue}. The resulting
+   * {@link WindowedValue} will be in no windows, and will have the default timestamp
+   * and pane.
+   *
+   * @param <T>   The type of the object.
+   * @return A function that accepts an object and returns its {@link WindowedValue}.
+   */
+  public static <T> Function<T, WindowedValue<T>> windowFunction() {
+    return new Function<T, WindowedValue<T>>() {
+      @Override
+      public WindowedValue<T> call(T t) {
+        return WindowedValue.valueInEmptyWindows(t);
+      }
+    };
+  }
+
+  /**
+   * A function for extracting the value from a {@link WindowedValue}.
+   *
+   * @param <T>   The type of the object.
+   * @return A function that accepts a {@link WindowedValue} and returns its value.
+   */
+  public static <T> Function<WindowedValue<T>, T> unwindowFunction() {
+    return new Function<WindowedValue<T>, T>() {
+      @Override
+      public T call(WindowedValue<T> t) {
+        return t.getValue();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
new file mode 100644
index 0000000..a82dbbe
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.aggregators;
+
+import org.apache.spark.AccumulatorParam;
+
+public class AggAccumParam implements AccumulatorParam<NamedAggregators> {
+  @Override
+  public NamedAggregators addAccumulator(NamedAggregators current, NamedAggregators added) {
+    return current.merge(added);
+  }
+
+  @Override
+  public NamedAggregators addInPlace(NamedAggregators current, NamedAggregators added) {
+    return addAccumulator(current, added);
+  }
+
+  @Override
+  public NamedAggregators zero(NamedAggregators initialValue) {
+    return new NamedAggregators();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
new file mode 100644
index 0000000..2747703
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
@@ -0,0 +1,202 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.aggregators;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.TreeMap;
+
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.beam.runners.spark.SparkRuntimeContext;
+
+/**
+ * This class wraps a map of named aggregators. Spark expects that all accumulators be declared
+ * before a job is launched. Dataflow allows aggregators to be used and incremented on the fly.
+ * We create a map of named aggregators and instantiate in the the spark context before the job
+ * is launched. We can then add aggregators on the fly in Spark.
+ */
+public class NamedAggregators implements Serializable {
+  /**
+   * Map from aggregator name to current state.
+   */
+  private final Map<String, State<?, ?, ?>> mNamedAggregators = new TreeMap<>();
+
+  /**
+   * Constructs a new NamedAggregators instance.
+   */
+  public NamedAggregators() {
+  }
+
+  /**
+   * Constructs a new named aggregators instance that contains a mapping from the specified
+   * `named` to the associated initial state.
+   *
+   * @param name  Name of aggregator.
+   * @param state Associated State.
+   */
+  public NamedAggregators(String name, State<?, ?, ?> state) {
+    this.mNamedAggregators.put(name, state);
+  }
+
+  /**
+   * @param name      Name of aggregator to retrieve.
+   * @param typeClass Type class to cast the value to.
+   * @param <T>       Type to be returned.
+   * @return the value of the aggregator associated with the specified name
+   */
+  public <T> T getValue(String name, Class<T> typeClass) {
+    return typeClass.cast(mNamedAggregators.get(name).render());
+  }
+
+  /**
+   * Merges another NamedAggregators instance with this instance.
+   *
+   * @param other The other instance of named aggregators ot merge.
+   * @return This instance of Named aggregators with associated states updated to reflect the
+   * other instance's aggregators.
+   */
+  public NamedAggregators merge(NamedAggregators other) {
+    for (Map.Entry<String, State<?, ?, ?>> e : other.mNamedAggregators.entrySet()) {
+      String key = e.getKey();
+      State<?, ?, ?> otherValue = e.getValue();
+      State<?, ?, ?> value = mNamedAggregators.get(key);
+      if (value == null) {
+        mNamedAggregators.put(key, otherValue);
+      } else {
+        mNamedAggregators.put(key, merge(value, otherValue));
+      }
+    }
+    return this;
+  }
+
+  /**
+   * Helper method to merge States whose generic types aren't provably the same,
+   * so require some casting.
+   */
+  @SuppressWarnings("unchecked")
+  private static <A, B, C> State<A, B, C> merge(State<?, ?, ?> s1, State<?, ?, ?> s2) {
+    return ((State<A, B, C>) s1).merge((State<A, B, C>) s2);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (Map.Entry<String, State<?, ?, ?>> e : mNamedAggregators.entrySet()) {
+      sb.append(e.getKey()).append(": ").append(e.getValue().render());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * @param <IN>    Input data type
+   * @param <INTER> Intermediate data type (useful for averages)
+   * @param <OUT>   Output data type
+   */
+  public interface State<IN, INTER, OUT> extends Serializable {
+    /**
+     * @param element new element to update state
+     */
+    void update(IN element);
+
+    State<IN, INTER, OUT> merge(State<IN, INTER, OUT> other);
+
+    INTER current();
+
+    OUT render();
+
+    Combine.CombineFn<IN, INTER, OUT> getCombineFn();
+  }
+
+  /**
+   * =&gt; combineFunction in data flow.
+   */
+  public static class CombineFunctionState<IN, INTER, OUT> implements State<IN, INTER, OUT> {
+
+    private Combine.CombineFn<IN, INTER, OUT> combineFn;
+    private Coder<IN> inCoder;
+    private SparkRuntimeContext ctxt;
+    private transient INTER state;
+
+    public CombineFunctionState(
+        Combine.CombineFn<IN, INTER, OUT> combineFn,
+        Coder<IN> inCoder,
+        SparkRuntimeContext ctxt) {
+      this.combineFn = combineFn;
+      this.inCoder = inCoder;
+      this.ctxt = ctxt;
+      this.state = combineFn.createAccumulator();
+    }
+
+    @Override
+    public void update(IN element) {
+      combineFn.addInput(state, element);
+    }
+
+    @Override
+    public State<IN, INTER, OUT> merge(State<IN, INTER, OUT> other) {
+      this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current()));
+      return this;
+    }
+
+    @Override
+    public INTER current() {
+      return state;
+    }
+
+    @Override
+    public OUT render() {
+      return combineFn.extractOutput(state);
+    }
+
+    @Override
+    public Combine.CombineFn<IN, INTER, OUT> getCombineFn() {
+      return combineFn;
+    }
+
+    private void writeObject(ObjectOutputStream oos) throws IOException {
+      oos.writeObject(ctxt);
+      oos.writeObject(combineFn);
+      oos.writeObject(inCoder);
+      try {
+        combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder)
+            .encode(state, oos, Coder.Context.NESTED);
+      } catch (CannotProvideCoderException e) {
+        throw new IllegalStateException("Could not determine coder for accumulator", e);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+      ctxt = (SparkRuntimeContext) ois.readObject();
+      combineFn = (Combine.CombineFn<IN, INTER, OUT>) ois.readObject();
+      inCoder = (Coder<IN>) ois.readObject();
+      try {
+        state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder)
+            .decode(ois, Coder.Context.NESTED);
+      } catch (CannotProvideCoderException e) {
+        throw new IllegalStateException("Could not determine coder for accumulator", e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
new file mode 100644
index 0000000..7d75e7d
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.coders;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.util.ByteArray;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Tuple2;
+
+/**
+ * Serialization utility class.
+ */
+public final class CoderHelpers {
+  private CoderHelpers() {
+  }
+
+  /**
+   * Utility method for serializing an object using the specified coder.
+   *
+   * @param value Value to serialize.
+   * @param coder Coder to serialize with.
+   * @param <T> type of value that is serialized
+   * @return Byte array representing serialized object.
+   */
+  public static <T> byte[] toByteArray(T value, Coder<T> coder) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      coder.encode(value, baos, new Coder.Context(true));
+    } catch (IOException e) {
+      throw new IllegalStateException("Error encoding value: " + value, e);
+    }
+    return baos.toByteArray();
+  }
+
+  /**
+   * Utility method for serializing a Iterable of values using the specified coder.
+   *
+   * @param values Values to serialize.
+   * @param coder  Coder to serialize with.
+   * @param <T> type of value that is serialized
+   * @return List of bytes representing serialized objects.
+   */
+  public static <T> List<byte[]> toByteArrays(Iterable<T> values, Coder<T> coder) {
+    List<byte[]> res = new LinkedList<>();
+    for (T value : values) {
+      res.add(toByteArray(value, coder));
+    }
+    return res;
+  }
+
+  /**
+   * Utility method for deserializing a byte array using the specified coder.
+   *
+   * @param serialized bytearray to be deserialized.
+   * @param coder      Coder to deserialize with.
+   * @param <T>        Type of object to be returned.
+   * @return Deserialized object.
+   */
+  public static <T> T fromByteArray(byte[] serialized, Coder<T> coder) {
+    ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+    try {
+      return coder.decode(bais, new Coder.Context(true));
+    } catch (IOException e) {
+      throw new IllegalStateException("Error decoding bytes for coder: " + coder, e);
+    }
+  }
+
+  /**
+   * A function wrapper for converting an object to a bytearray.
+   *
+   * @param coder Coder to serialize with.
+   * @param <T>   The type of the object being serialized.
+   * @return A function that accepts an object and returns its coder-serialized form.
+   */
+  public static <T> Function<T, byte[]> toByteFunction(final Coder<T> coder) {
+    return new Function<T, byte[]>() {
+      @Override
+      public byte[] call(T t) throws Exception {
+        return toByteArray(t, coder);
+      }
+    };
+  }
+
+  /**
+   * A function wrapper for converting a byte array to an object.
+   *
+   * @param coder Coder to deserialize with.
+   * @param <T>   The type of the object being deserialized.
+   * @return A function that accepts a byte array and returns its corresponding object.
+   */
+  public static <T> Function<byte[], T> fromByteFunction(final Coder<T> coder) {
+    return new Function<byte[], T>() {
+      @Override
+      public T call(byte[] bytes) throws Exception {
+        return fromByteArray(bytes, coder);
+      }
+    };
+  }
+
+  /**
+   * A function wrapper for converting a key-value pair to a byte array pair.
+   *
+   * @param keyCoder Coder to serialize keys.
+   * @param valueCoder Coder to serialize values.
+   * @param <K>   The type of the key being serialized.
+   * @param <V>   The type of the value being serialized.
+   * @return A function that accepts a key-value pair and returns a pair of byte arrays.
+   */
+  public static <K, V> PairFunction<Tuple2<K, V>, ByteArray, byte[]> toByteFunction(
+      final Coder<K> keyCoder, final Coder<V> valueCoder) {
+    return new PairFunction<Tuple2<K, V>, ByteArray, byte[]>() {
+      @Override
+      public Tuple2<ByteArray, byte[]> call(Tuple2<K, V> kv) {
+        return new Tuple2<>(new ByteArray(toByteArray(kv._1(), keyCoder)), toByteArray(kv._2(),
+            valueCoder));
+      }
+    };
+  }
+
+  /**
+   * A function wrapper for converting a byte array pair to a key-value pair.
+   *
+   * @param keyCoder Coder to deserialize keys.
+   * @param valueCoder Coder to deserialize values.
+   * @param <K>   The type of the key being deserialized.
+   * @param <V>   The type of the value being deserialized.
+   * @return A function that accepts a pair of byte arrays and returns a key-value pair.
+   */
+  public static <K, V> PairFunction<Tuple2<ByteArray, byte[]>, K, V> fromByteFunction(
+      final Coder<K> keyCoder, final Coder<V> valueCoder) {
+    return new PairFunction<Tuple2<ByteArray, byte[]>, K, V>() {
+      @Override
+      public Tuple2<K, V> call(Tuple2<ByteArray, byte[]> tuple) {
+        return new Tuple2<>(fromByteArray(tuple._1().getValue(), keyCoder),
+            fromByteArray(tuple._2(), valueCoder));
+      }
+    };
+  }
+
+  /**
+   * A function wrapper for converting a byte array pair to a key-value pair, where
+   * values are {@link Iterable}.
+   *
+   * @param keyCoder Coder to deserialize keys.
+   * @param valueCoder Coder to deserialize values.
+   * @param <K>   The type of the key being deserialized.
+   * @param <V>   The type of the value being deserialized.
+   * @return A function that accepts a pair of byte arrays and returns a key-value pair.
+   */
+  public static <K, V> PairFunction<Tuple2<ByteArray, Iterable<byte[]>>, K, Iterable<V>>
+      fromByteFunctionIterable(final Coder<K> keyCoder, final Coder<V> valueCoder) {
+    return new PairFunction<Tuple2<ByteArray, Iterable<byte[]>>, K, Iterable<V>>() {
+      @Override
+      public Tuple2<K, Iterable<V>> call(Tuple2<ByteArray, Iterable<byte[]>> tuple) {
+        return new Tuple2<>(fromByteArray(tuple._1().getValue(), keyCoder),
+          Iterables.transform(tuple._2(), new com.google.common.base.Function<byte[], V>() {
+            @Override
+            public V apply(byte[] bytes) {
+              return fromByteArray(bytes, valueCoder);
+            }
+          }));
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
new file mode 100644
index 0000000..5b77e97
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.coders;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.hadoop.io.NullWritable;
+
+public final class NullWritableCoder extends WritableCoder<NullWritable> {
+  private static final long serialVersionUID = 1L;
+
+  @JsonCreator
+  public static NullWritableCoder of() {
+    return INSTANCE;
+  }
+
+  private static final NullWritableCoder INSTANCE = new NullWritableCoder();
+
+  private NullWritableCoder() {
+    super(NullWritable.class);
+  }
+
+  @Override
+  public void encode(NullWritable value, OutputStream outStream, Context context) {
+    // nothing to write
+  }
+
+  @Override
+  public NullWritable decode(InputStream inStream, Context context) {
+    return NullWritable.get();
+  }
+
+  @Override
+  public boolean consistentWithEquals() {
+    return true;
+  }
+
+  /**
+   * Returns true since registerByteSizeObserver() runs in constant time.
+   */
+  @Override
+  public boolean isRegisterByteSizeObserverCheap(NullWritable value, Context context) {
+    return true;
+  }
+
+  @Override
+  protected long getEncodedElementByteSize(NullWritable value, Context context) {
+    return 0;
+  }
+
+  @Override
+  public void verifyDeterministic() throws Coder.NonDeterministicException {
+    // NullWritableCoder is deterministic
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
new file mode 100644
index 0000000..fa73753
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.coders;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderException;
+import com.google.cloud.dataflow.sdk.coders.StandardCoder;
+import com.google.cloud.dataflow.sdk.util.CloudObject;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}.
+ *
+ * <p> To use, specify the coder type on a PCollection:
+ * <pre>
+ * {@code
+ *   PCollection<MyRecord> records =
+ *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
+ * }
+ * </pre>
+ *
+ * @param <T> the type of elements handled by this coder
+ */
+public class WritableCoder<T extends Writable> extends StandardCoder<T> {
+  private static final long serialVersionUID = 0L;
+
+  /**
+   * Returns a {@code WritableCoder} instance for the provided element class.
+   * @param <T> the element type
+   * @param clazz the element class
+   * @return a {@code WritableCoder} instance for the provided element class
+   */
+  public static <T extends Writable> WritableCoder<T> of(Class<T> clazz) {
+    if (clazz.equals(NullWritable.class)) {
+      @SuppressWarnings("unchecked")
+      WritableCoder<T> result = (WritableCoder<T>) NullWritableCoder.of();
+      return result;
+    }
+    return new WritableCoder<>(clazz);
+  }
+
+  @JsonCreator
+  @SuppressWarnings("unchecked")
+  public static WritableCoder<?> of(@JsonProperty("type") String classType)
+      throws ClassNotFoundException {
+    Class<?> clazz = Class.forName(classType);
+    if (!Writable.class.isAssignableFrom(clazz)) {
+      throw new ClassNotFoundException(
+          "Class " + classType + " does not implement Writable");
+    }
+    return of((Class<? extends Writable>) clazz);
+  }
+
+  private final Class<T> type;
+
+  public WritableCoder(Class<T> type) {
+    this.type = type;
+  }
+
+  @Override
+  public void encode(T value, OutputStream outStream, Context context) throws IOException {
+    value.write(new DataOutputStream(outStream));
+  }
+
+  @Override
+  public T decode(InputStream inStream, Context context) throws IOException {
+    try {
+      T t = type.getConstructor().newInstance();
+      t.readFields(new DataInputStream(inStream));
+      return t;
+    } catch (NoSuchMethodException | InstantiationException | IllegalAccessException e) {
+      throw new CoderException("unable to deserialize record", e);
+    } catch (InvocationTargetException ite) {
+      throw new CoderException("unable to deserialize record", ite.getCause());
+    }
+  }
+
+  @Override
+  public List<Coder<?>> getCoderArguments() {
+    return null;
+  }
+
+  @Override
+  public CloudObject asCloudObject() {
+    CloudObject result = super.asCloudObject();
+    result.put("type", type.getName());
+    return result;
+  }
+
+  @Override
+  public void verifyDeterministic() throws Coder.NonDeterministicException {
+    throw new NonDeterministicException(this,
+        "Hadoop Writable may be non-deterministic.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
new file mode 100644
index 0000000..2ee072a
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.io;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+/**
+ * Print to console.
+ */
+public final class ConsoleIO {
+
+  private ConsoleIO() {
+  }
+
+  public static final class Write {
+
+    private Write() {
+    }
+
+    public static <T> Unbound<T> from() {
+      return new Unbound<>(10);
+    }
+
+    public static <T> Unbound<T> from(int num) {
+      return new Unbound<>(num);
+    }
+
+    public static class Unbound<T> extends PTransform<PCollection<T>, PDone> {
+
+      private final int num;
+
+      Unbound(int num) {
+        this.num = num;
+      }
+
+      public int getNum() {
+        return num;
+      }
+
+      @Override
+      public PDone apply(PCollection<T> input) {
+        return PDone.in(input.getPipeline());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
new file mode 100644
index 0000000..c92f8bf
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.io;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.common.base.Preconditions;
+
+/**
+ * Create an input stream from Queue.
+ *
+ * @param <T> stream type
+ */
+public final class CreateStream<T> {
+
+  private CreateStream() {
+  }
+
+  /**
+   * Define the input stream to create from queue.
+   *
+   * @param queuedValues  defines the input stream
+   * @param <T>           stream type
+   * @return the queue that defines the input stream
+   */
+  public static <T> QueuedValues<T> fromQueue(Iterable<Iterable<T>> queuedValues) {
+    return new QueuedValues<>(queuedValues);
+  }
+
+  public static final class QueuedValues<T> extends PTransform<PInput, PCollection<T>> {
+
+    private final Iterable<Iterable<T>> queuedValues;
+
+    QueuedValues(Iterable<Iterable<T>> queuedValues) {
+      Preconditions.checkNotNull(queuedValues,
+              "need to set the queuedValues of an Create.QueuedValues transform");
+      this.queuedValues = queuedValues;
+    }
+
+    public Iterable<Iterable<T>> getQueuedValues() {
+      return queuedValues;
+    }
+
+    @Override
+    public PCollection<T> apply(PInput input) {
+      // Spark streaming micro batches are bounded by default
+      return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
+          WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
new file mode 100644
index 0000000..9798157
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.io;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.common.base.Preconditions;
+
+import kafka.serializer.Decoder;
+
+/**
+ * Read stream from Kafka.
+ */
+public final class KafkaIO {
+
+  private KafkaIO() {
+  }
+
+  public static final class Read {
+
+    private Read() {
+    }
+
+    /**
+     * Define the Kafka consumption.
+     *
+     * @param keyDecoder    {@link Decoder} to decode the Kafka message key
+     * @param valueDecoder  {@link Decoder} to decode the Kafka message value
+     * @param key           Kafka message key Class
+     * @param value         Kafka message value Class
+     * @param topics        Kafka topics to subscribe
+     * @param kafkaParams   map of Kafka parameters
+     * @param <K>           Kafka message key Class type
+     * @param <V>           Kafka message value Class type
+     * @return KafkaIO Unbound input
+     */
+    public static <K, V> Unbound<K, V> from(Class<? extends Decoder<K>> keyDecoder,
+                                            Class<? extends Decoder<V>> valueDecoder,
+                                            Class<K> key,
+                                            Class<V> value, Set<String> topics,
+                                            Map<String, String> kafkaParams) {
+      return new Unbound<>(keyDecoder, valueDecoder, key, value, topics, kafkaParams);
+    }
+
+    public static class Unbound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
+
+      private final Class<? extends Decoder<K>> keyDecoderClass;
+      private final Class<? extends Decoder<V>> valueDecoderClass;
+      private final Class<K> keyClass;
+      private final Class<V> valueClass;
+      private final Set<String> topics;
+      private final Map<String, String> kafkaParams;
+
+      Unbound(Class<? extends Decoder<K>> keyDecoder,
+              Class<? extends Decoder<V>> valueDecoder, Class<K> key,
+              Class<V> value, Set<String> topics, Map<String, String> kafkaParams) {
+        Preconditions.checkNotNull(keyDecoder,
+            "need to set the key decoder class of a KafkaIO.Read transform");
+        Preconditions.checkNotNull(valueDecoder,
+            "need to set the value decoder class of a KafkaIO.Read transform");
+        Preconditions.checkNotNull(key,
+            "need to set the key class of aKafkaIO.Read transform");
+        Preconditions.checkNotNull(value,
+            "need to set the value class of a KafkaIO.Read transform");
+        Preconditions.checkNotNull(topics,
+            "need to set the topics of a KafkaIO.Read transform");
+        Preconditions.checkNotNull(kafkaParams,
+            "need to set the kafkaParams of a KafkaIO.Read transform");
+        this.keyDecoderClass = keyDecoder;
+        this.valueDecoderClass = valueDecoder;
+        this.keyClass = key;
+        this.valueClass = value;
+        this.topics = topics;
+        this.kafkaParams = kafkaParams;
+      }
+
+      public Class<? extends Decoder<K>> getKeyDecoderClass() {
+        return keyDecoderClass;
+      }
+
+      public Class<? extends Decoder<V>> getValueDecoderClass() {
+        return valueDecoderClass;
+      }
+
+      public Class<V> getValueClass() {
+        return valueClass;
+      }
+
+      public Class<K> getKeyClass() {
+        return keyClass;
+      }
+
+      public Set<String> getTopics() {
+        return topics;
+      }
+
+      public Map<String, String> getKafkaParams() {
+        return kafkaParams;
+      }
+
+      @Override
+      public PCollection<KV<K, V>> apply(PInput input) {
+        // Spark streaming micro batches are bounded by default
+        return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
+            WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
new file mode 100644
index 0000000..e8d2aa1
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+public final class HadoopIO {
+
+  private HadoopIO() {
+  }
+
+  public static final class Read {
+
+    private Read() {
+    }
+
+    public static <K, V> Bound<K, V> from(String filepattern,
+        Class<? extends FileInputFormat<K, V>> format, Class<K> key, Class<V> value) {
+      return new Bound<>(filepattern, format, key, value);
+    }
+
+    public static class Bound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
+
+      private final String filepattern;
+      private final Class<? extends FileInputFormat<K, V>> formatClass;
+      private final Class<K> keyClass;
+      private final Class<V> valueClass;
+
+      Bound(String filepattern, Class<? extends FileInputFormat<K, V>> format, Class<K> key,
+          Class<V> value) {
+        Preconditions.checkNotNull(filepattern,
+                                   "need to set the filepattern of an HadoopIO.Read transform");
+        Preconditions.checkNotNull(format,
+                                   "need to set the format class of an HadoopIO.Read transform");
+        Preconditions.checkNotNull(key,
+                                   "need to set the key class of an HadoopIO.Read transform");
+        Preconditions.checkNotNull(value,
+                                   "need to set the value class of an HadoopIO.Read transform");
+        this.filepattern = filepattern;
+        this.formatClass = format;
+        this.keyClass = key;
+        this.valueClass = value;
+      }
+
+      public String getFilepattern() {
+        return filepattern;
+      }
+
+      public Class<? extends FileInputFormat<K, V>> getFormatClass() {
+        return formatClass;
+      }
+
+      public Class<V> getValueClass() {
+        return valueClass;
+      }
+
+      public Class<K> getKeyClass() {
+        return keyClass;
+      }
+
+      @Override
+      public PCollection<KV<K, V>> apply(PInput input) {
+        return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
+            WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED);
+      }
+
+    }
+
+  }
+
+  public static final class Write {
+
+    private Write() {
+    }
+
+    public static <K, V> Bound<K, V> to(String filenamePrefix,
+        Class<? extends FileOutputFormat<K, V>> format, Class<K> key, Class<V> value) {
+      return new Bound<>(filenamePrefix, format, key, value);
+    }
+
+    public static class Bound<K, V> extends PTransform<PCollection<KV<K, V>>, PDone> {
+
+      /** The filename to write to. */
+      private final String filenamePrefix;
+      /** Suffix to use for each filename. */
+      private final String filenameSuffix;
+      /** Requested number of shards.  0 for automatic. */
+      private final int numShards;
+      /** Shard template string. */
+      private final String shardTemplate;
+      private final Class<? extends FileOutputFormat<K, V>> formatClass;
+      private final Class<K> keyClass;
+      private final Class<V> valueClass;
+      private final Map<String, String> configurationProperties;
+
+      Bound(String filenamePrefix, Class<? extends FileOutputFormat<K, V>> format,
+          Class<K> key,
+          Class<V> value) {
+        this(filenamePrefix, "", 0, ShardNameTemplate.INDEX_OF_MAX, format, key, value,
+            new HashMap<String, String>());
+      }
+
+      Bound(String filenamePrefix, String filenameSuffix, int numShards,
+          String shardTemplate, Class<? extends FileOutputFormat<K, V>> format,
+          Class<K> key, Class<V> value, Map<String, String> configurationProperties) {
+        this.filenamePrefix = filenamePrefix;
+        this.filenameSuffix = filenameSuffix;
+        this.numShards = numShards;
+        this.shardTemplate = shardTemplate;
+        this.formatClass = format;
+        this.keyClass = key;
+        this.valueClass = value;
+        this.configurationProperties = configurationProperties;
+      }
+
+      public Bound<K, V> withoutSharding() {
+        return new Bound<>(filenamePrefix, filenameSuffix, 1, "", formatClass,
+            keyClass, valueClass, configurationProperties);
+      }
+
+      public Bound<K, V> withConfigurationProperty(String key, String value) {
+        configurationProperties.put(key, value);
+        return this;
+      }
+
+      public String getFilenamePrefix() {
+        return filenamePrefix;
+      }
+
+      public String getShardTemplate() {
+        return shardTemplate;
+      }
+
+      public int getNumShards() {
+        return numShards;
+      }
+
+      public String getFilenameSuffix() {
+        return filenameSuffix;
+      }
+
+      public Class<? extends FileOutputFormat<K, V>> getFormatClass() {
+        return formatClass;
+      }
+
+      public Class<V> getValueClass() {
+        return valueClass;
+      }
+
+      public Class<K> getKeyClass() {
+        return keyClass;
+      }
+
+      public Map<String, String> getConfigurationProperties() {
+        return configurationProperties;
+      }
+
+      @Override
+      public PDone apply(PCollection<KV<K, V>> input) {
+        Preconditions.checkNotNull(filenamePrefix,
+            "need to set the filename prefix of an HadoopIO.Write transform");
+        Preconditions.checkNotNull(formatClass,
+            "need to set the format class of an HadoopIO.Write transform");
+        Preconditions.checkNotNull(keyClass,
+            "need to set the key class of an HadoopIO.Write transform");
+        Preconditions.checkNotNull(valueClass,
+            "need to set the value class of an HadoopIO.Write transform");
+
+        Preconditions.checkArgument(ShardNameTemplateAware.class.isAssignableFrom(formatClass),
+            "Format class must implement " + ShardNameTemplateAware.class.getName());
+
+        return PDone.in(input.getPipeline());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
new file mode 100644
index 0000000..21c7985
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.Path;
+
+public final class ShardNameBuilder {
+
+  private ShardNameBuilder() {
+  }
+
+  /**
+   * Replace occurrences of uppercase letters 'N' with the given {code}shardCount{code},
+   * left-padded with zeros if necessary.
+   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
+   * @param template the string template containing uppercase letters 'N'
+   * @param shardCount the total number of shards
+   * @return a string template with 'N' replaced by the shard count
+   */
+  public static String replaceShardCount(String template, int shardCount) {
+    return replaceShardPattern(template, "N+", shardCount);
+  }
+
+  /**
+   * Replace occurrences of uppercase letters 'S' with the given {code}shardNumber{code},
+   * left-padded with zeros if necessary.
+   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
+   * @param template the string template containing uppercase letters 'S'
+   * @param shardNumber the number of a particular shard
+   * @return a string template with 'S' replaced by the shard number
+   */
+  public static String replaceShardNumber(String template, int shardNumber) {
+    return replaceShardPattern(template, "S+", shardNumber);
+  }
+
+  private static String replaceShardPattern(String template, String pattern, int n) {
+    Pattern p = Pattern.compile(pattern);
+    Matcher m = p.matcher(template);
+    StringBuffer sb = new StringBuffer();
+    while (m.find()) {
+      // replace pattern with a String format string:
+      // index 1, zero-padding flag (0), width length of matched pattern, decimal conversion
+      m.appendReplacement(sb, "%1\\$0" + m.group().length() + "d");
+    }
+    m.appendTail(sb);
+    return String.format(sb.toString(), n);
+  }
+
+  /**
+   * @param pathPrefix a relative or absolute path
+   * @param template a template string
+   * @return the output directory for the given prefix, template and suffix
+   */
+  public static String getOutputDirectory(String pathPrefix, String template) {
+    String out = new Path(pathPrefix + template).getParent().toString();
+    if (out.isEmpty()) {
+      return "./";
+    }
+    return out;
+  }
+
+  /**
+   * @param pathPrefix a relative or absolute path
+   * @param template a template string
+   * @return the prefix of the output filename for the given path prefix and template
+   */
+  public static String getOutputFilePrefix(String pathPrefix, String template) {
+    String name = new Path(pathPrefix + template).getName();
+    if (name.endsWith(template)) {
+      return name.substring(0, name.length() - template.length());
+    } else {
+      return "";
+    }
+  }
+
+  /**
+   * @param pathPrefix a relative or absolute path
+   * @param template a template string
+   * @return the template for the output filename for the given path prefix and
+   * template
+   */
+  public static String getOutputFileTemplate(String pathPrefix, String template) {
+    String name = new Path(pathPrefix + template).getName();
+    if (name.endsWith(template)) {
+      return template;
+    } else {
+      return name;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
new file mode 100644
index 0000000..fdee42b
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+/**
+ * A marker interface that implementations of
+ * {@link org.apache.hadoop.mapreduce.lib.output.FileOutputFormat} implement to indicate
+ * that they produce shard names that adhere to the template in
+ * {@link HadoopIO.Write}.
+ *
+ * Some common shard names are defined in
+ * {@link com.google.cloud.dataflow.sdk.io.ShardNameTemplate}.
+ */
+public interface ShardNameTemplateAware {
+}


[18/23] incubator-beam git commit: [BEAM-11] second iteration of package reorganisation

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
deleted file mode 100644
index 6a9dca0..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
+++ /dev/null
@@ -1,808 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.ImmutableMap;
-
-import org.apache.avro.mapred.AvroKey;
-import org.apache.avro.mapreduce.AvroJob;
-import org.apache.avro.mapreduce.AvroKeyInputFormat;
-import org.apache.beam.runners.spark.coders.CoderHelpers;
-import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
-import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper;
-import org.apache.beam.runners.spark.io.hadoop.TemplatedAvroKeyOutputFormat;
-import org.apache.beam.runners.spark.io.hadoop.TemplatedTextOutputFormat;
-import org.apache.beam.runners.spark.util.BroadcastHelper;
-import org.apache.beam.runners.spark.util.ByteArray;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.Function2;
-import org.apache.spark.api.java.function.PairFunction;
-import scala.Tuple2;
-
-/**
- * Supports translation between a DataFlow transform, and Spark's operations on RDDs.
- */
-public final class TransformTranslator {
-
-  private TransformTranslator() {
-  }
-
-  public static class FieldGetter {
-    private final Map<String, Field> fields;
-
-    public FieldGetter(Class<?> clazz) {
-      this.fields = Maps.newHashMap();
-      for (Field f : clazz.getDeclaredFields()) {
-        f.setAccessible(true);
-        this.fields.put(f.getName(), f);
-      }
-    }
-
-    public <T> T get(String fieldname, Object value) {
-      try {
-        @SuppressWarnings("unchecked")
-        T fieldValue = (T) fields.get(fieldname).get(value);
-        return fieldValue;
-      } catch (IllegalAccessException e) {
-        throw new IllegalStateException(e);
-      }
-    }
-  }
-
-  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
-    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
-        PCollectionList<T> pcs = context.getInput(transform);
-        JavaRDD<WindowedValue<T>>[] rdds = new JavaRDD[pcs.size()];
-        for (int i = 0; i < rdds.length; i++) {
-          rdds[i] = (JavaRDD<WindowedValue<T>>) context.getRDD(pcs.get(i));
-        }
-        JavaRDD<WindowedValue<T>> rdd = context.getSparkContext().union(rdds);
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>> gbk() {
-    return new TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>>() {
-      @Override
-      public void evaluate(GroupByKey.GroupByKeyOnly<K, V> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, V>>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context.getInputRDD(transform);
-        @SuppressWarnings("unchecked")
-        KvCoder<K, V> coder = (KvCoder<K, V>) context.getInput(transform).getCoder();
-        Coder<K> keyCoder = coder.getKeyCoder();
-        Coder<V> valueCoder = coder.getValueCoder();
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaRDDLike<WindowedValue<KV<K, Iterable<V>>>, ?> outRDD = fromPair(
-              toPair(inRDD.map(WindowingHelpers.<KV<K, V>>unwindowFunction()))
-            .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder))
-            .groupByKey()
-            .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder)))
-            // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
-            .map(WindowingHelpers.<KV<K, Iterable<V>>>windowFunction());
-        context.setOutputRDD(transform, outRDD);
-      }
-    };
-  }
-
-  private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class);
-
-  private static <K, VI, VO> TransformEvaluator<Combine.GroupedValues<K, VI, VO>> grouped() {
-    return new TransformEvaluator<Combine.GroupedValues<K, VI, VO>>() {
-      @Override
-      public void evaluate(Combine.GroupedValues<K, VI, VO> transform, EvaluationContext context) {
-        Combine.KeyedCombineFn<K, VI, ?, VO> keyed = GROUPED_FG.get("fn", transform);
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?>) context.getInputRDD(transform);
-        context.setOutputRDD(transform,
-            inRDD.map(new KVFunction<>(keyed)));
-      }
-    };
-  }
-
-  private static final FieldGetter COMBINE_GLOBALLY_FG = new FieldGetter(Combine.Globally.class);
-
-  private static <I, A, O> TransformEvaluator<Combine.Globally<I, O>> combineGlobally() {
-    return new TransformEvaluator<Combine.Globally<I, O>>() {
-
-      @Override
-      public void evaluate(Combine.Globally<I, O> transform, EvaluationContext context) {
-        final Combine.CombineFn<I, A, O> globally = COMBINE_GLOBALLY_FG.get("fn", transform);
-
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRdd =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-
-        final Coder<I> iCoder = context.getInput(transform).getCoder();
-        final Coder<A> aCoder;
-        try {
-          aCoder = globally.getAccumulatorCoder(
-              context.getPipeline().getCoderRegistry(), iCoder);
-        } catch (CannotProvideCoderException e) {
-          throw new IllegalStateException("Could not determine coder for accumulator", e);
-        }
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaRDD<byte[]> inRddBytes = inRdd
-            .map(WindowingHelpers.<I>unwindowFunction())
-            .map(CoderHelpers.toByteFunction(iCoder));
-
-        /*A*/ byte[] acc = inRddBytes.aggregate(
-            CoderHelpers.toByteArray(globally.createAccumulator(), aCoder),
-            new Function2</*A*/ byte[], /*I*/ byte[], /*A*/ byte[]>() {
-              @Override
-              public /*A*/ byte[] call(/*A*/ byte[] ab, /*I*/ byte[] ib) throws Exception {
-                A a = CoderHelpers.fromByteArray(ab, aCoder);
-                I i = CoderHelpers.fromByteArray(ib, iCoder);
-                return CoderHelpers.toByteArray(globally.addInput(a, i), aCoder);
-              }
-            },
-            new Function2</*A*/ byte[], /*A*/ byte[], /*A*/ byte[]>() {
-              @Override
-              public /*A*/ byte[] call(/*A*/ byte[] a1b, /*A*/ byte[] a2b) throws Exception {
-                A a1 = CoderHelpers.fromByteArray(a1b, aCoder);
-                A a2 = CoderHelpers.fromByteArray(a2b, aCoder);
-                // don't use Guava's ImmutableList.of as values may be null
-                List<A> accumulators = Collections.unmodifiableList(Arrays.asList(a1, a2));
-                A merged = globally.mergeAccumulators(accumulators);
-                return CoderHelpers.toByteArray(merged, aCoder);
-              }
-            }
-        );
-        O output = globally.extractOutput(CoderHelpers.fromByteArray(acc, aCoder));
-
-        Coder<O> coder = context.getOutput(transform).getCoder();
-        JavaRDD<byte[]> outRdd = context.getSparkContext().parallelize(
-            // don't use Guava's ImmutableList.of as output may be null
-            CoderHelpers.toByteArrays(Collections.singleton(output), coder));
-        context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder))
-            .map(WindowingHelpers.<O>windowFunction()));
-      }
-    };
-  }
-
-  private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class);
-
-  private static <K, VI, VA, VO> TransformEvaluator<Combine.PerKey<K, VI, VO>> combinePerKey() {
-    return new TransformEvaluator<Combine.PerKey<K, VI, VO>>() {
-      @Override
-      public void evaluate(Combine.PerKey<K, VI, VO> transform, EvaluationContext context) {
-        final Combine.KeyedCombineFn<K, VI, VA, VO> keyed =
-            COMBINE_PERKEY_FG.get("fn", transform);
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, VI>>, ?> inRdd =
-            (JavaRDDLike<WindowedValue<KV<K, VI>>, ?>) context.getInputRDD(transform);
-
-        @SuppressWarnings("unchecked")
-        KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
-        Coder<K> keyCoder = inputCoder.getKeyCoder();
-        Coder<VI> viCoder = inputCoder.getValueCoder();
-        Coder<VA> vaCoder;
-        try {
-          vaCoder = keyed.getAccumulatorCoder(
-              context.getPipeline().getCoderRegistry(), keyCoder, viCoder);
-        } catch (CannotProvideCoderException e) {
-          throw new IllegalStateException("Could not determine coder for accumulator", e);
-        }
-        Coder<KV<K, VI>> kviCoder = KvCoder.of(keyCoder, viCoder);
-        Coder<KV<K, VA>> kvaCoder = KvCoder.of(keyCoder, vaCoder);
-
-        // We need to duplicate K as both the key of the JavaPairRDD as well as inside the value,
-        // since the functions passed to combineByKey don't receive the associated key of each
-        // value, and we need to map back into methods in Combine.KeyedCombineFn, which each
-        // require the key in addition to the VI's and VA's being merged/accumulated. Once Spark
-        // provides a way to include keys in the arguments of combine/merge functions, we won't
-        // need to duplicate the keys anymore.
-
-        // Key has to bw windowed in order to group by window as well
-        JavaPairRDD<WindowedValue<K>, WindowedValue<KV<K, VI>>> inRddDuplicatedKeyPair =
-            inRdd.mapToPair(
-                new PairFunction<WindowedValue<KV<K, VI>>, WindowedValue<K>,
-                    WindowedValue<KV<K, VI>>>() {
-                  @Override
-                  public Tuple2<WindowedValue<K>,
-                      WindowedValue<KV<K, VI>>> call(WindowedValue<KV<K, VI>> kv) {
-                    WindowedValue<K> wk = WindowedValue.of(kv.getValue().getKey(),
-                        kv.getTimestamp(), kv.getWindows(), kv.getPane());
-                    return new Tuple2<>(wk, kv);
-                  }
-                });
-        //-- windowed coders
-        final WindowedValue.FullWindowedValueCoder<K> wkCoder =
-                WindowedValue.FullWindowedValueCoder.of(keyCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-        final WindowedValue.FullWindowedValueCoder<KV<K, VI>> wkviCoder =
-                WindowedValue.FullWindowedValueCoder.of(kviCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-        final WindowedValue.FullWindowedValueCoder<KV<K, VA>> wkvaCoder =
-                WindowedValue.FullWindowedValueCoder.of(kvaCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaPairRDD<ByteArray, byte[]> inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair
-            .mapToPair(CoderHelpers.toByteFunction(wkCoder, wkviCoder));
-
-        // The output of combineByKey will be "VA" (accumulator) types rather than "VO" (final
-        // output types) since Combine.CombineFn only provides ways to merge VAs, and no way
-        // to merge VOs.
-        JavaPairRDD</*K*/ ByteArray, /*KV<K, VA>*/ byte[]> accumulatedBytes =
-            inRddDuplicatedKeyPairBytes.combineByKey(
-            new Function</*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VI>*/ byte[] input) {
-                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
-                VA va = keyed.createAccumulator(wkvi.getValue().getKey());
-                va = keyed.addInput(wkvi.getValue().getKey(), va, wkvi.getValue().getValue());
-                WindowedValue<KV<K, VA>> wkva =
-                    WindowedValue.of(KV.of(wkvi.getValue().getKey(), va), wkvi.getTimestamp(),
-                    wkvi.getWindows(), wkvi.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            },
-            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc,
-                  /*KV<K, VI>*/ byte[] input) {
-                WindowedValue<KV<K, VA>> wkva = CoderHelpers.fromByteArray(acc, wkvaCoder);
-                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
-                VA va = keyed.addInput(wkva.getValue().getKey(), wkva.getValue().getValue(),
-                    wkvi.getValue().getValue());
-                wkva = WindowedValue.of(KV.of(wkva.getValue().getKey(), va), wkva.getTimestamp(),
-                    wkva.getWindows(), wkva.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            },
-            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc1,
-                  /*KV<K, VA>*/ byte[] acc2) {
-                WindowedValue<KV<K, VA>> wkva1 = CoderHelpers.fromByteArray(acc1, wkvaCoder);
-                WindowedValue<KV<K, VA>> wkva2 = CoderHelpers.fromByteArray(acc2, wkvaCoder);
-                VA va = keyed.mergeAccumulators(wkva1.getValue().getKey(),
-                    // don't use Guava's ImmutableList.of as values may be null
-                    Collections.unmodifiableList(Arrays.asList(wkva1.getValue().getValue(),
-                    wkva2.getValue().getValue())));
-                WindowedValue<KV<K, VA>> wkva = WindowedValue.of(KV.of(wkva1.getValue().getKey(),
-                    va), wkva1.getTimestamp(), wkva1.getWindows(), wkva1.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            });
-
-        JavaPairRDD<WindowedValue<K>, WindowedValue<VO>> extracted = accumulatedBytes
-            .mapToPair(CoderHelpers.fromByteFunction(wkCoder, wkvaCoder))
-            .mapValues(
-                new Function<WindowedValue<KV<K, VA>>, WindowedValue<VO>>() {
-                  @Override
-                  public WindowedValue<VO> call(WindowedValue<KV<K, VA>> acc) {
-                    return WindowedValue.of(keyed.extractOutput(acc.getValue().getKey(),
-                        acc.getValue().getValue()), acc.getTimestamp(),
-                        acc.getWindows(), acc.getPane());
-                  }
-                });
-
-        context.setOutputRDD(transform,
-            fromPair(extracted)
-            .map(new Function<KV<WindowedValue<K>, WindowedValue<VO>>, WindowedValue<KV<K, VO>>>() {
-              @Override
-              public WindowedValue<KV<K, VO>> call(KV<WindowedValue<K>, WindowedValue<VO>> kwvo)
-                  throws Exception {
-                WindowedValue<VO> wvo = kwvo.getValue();
-                KV<K, VO> kvo = KV.of(kwvo.getKey().getValue(), wvo.getValue());
-                return WindowedValue.of(kvo, wvo.getTimestamp(), wvo.getWindows(), wvo.getPane());
-              }
-            }));
-      }
-    };
-  }
-
-  private static final class KVFunction<K, VI, VO>
-      implements Function<WindowedValue<KV<K, Iterable<VI>>>, WindowedValue<KV<K, VO>>> {
-    private final Combine.KeyedCombineFn<K, VI, ?, VO> keyed;
-
-     KVFunction(Combine.KeyedCombineFn<K, VI, ?, VO> keyed) {
-      this.keyed = keyed;
-    }
-
-    @Override
-    public WindowedValue<KV<K, VO>> call(WindowedValue<KV<K, Iterable<VI>>> windowedKv)
-        throws Exception {
-      KV<K, Iterable<VI>> kv = windowedKv.getValue();
-      return WindowedValue.of(KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())),
-          windowedKv.getTimestamp(), windowedKv.getWindows(), windowedKv.getPane());
-    }
-  }
-
-  private static <K, V> JavaPairRDD<K, V> toPair(JavaRDDLike<KV<K, V>, ?> rdd) {
-    return rdd.mapToPair(new PairFunction<KV<K, V>, K, V>() {
-      @Override
-      public Tuple2<K, V> call(KV<K, V> kv) {
-        return new Tuple2<>(kv.getKey(), kv.getValue());
-      }
-    });
-  }
-
-  private static <K, V> JavaRDDLike<KV<K, V>, ?> fromPair(JavaPairRDD<K, V> rdd) {
-    return rdd.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-      @Override
-      public KV<K, V> call(Tuple2<K, V> t2) {
-        return KV.of(t2._1(), t2._2());
-      }
-    });
-  }
-
-  private static <I, O> TransformEvaluator<ParDo.Bound<I, O>> parDo() {
-    return new TransformEvaluator<ParDo.Bound<I, O>>() {
-      @Override
-      public void evaluate(ParDo.Bound<I, O> transform, EvaluationContext context) {
-        DoFnFunction<I, O> dofn =
-            new DoFnFunction<>(transform.getFn(),
-                context.getRuntimeContext(),
-                getSideInputs(transform.getSideInputs(), context));
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-        context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
-      }
-    };
-  }
-
-  private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class);
-
-  private static <I, O> TransformEvaluator<ParDo.BoundMulti<I, O>> multiDo() {
-    return new TransformEvaluator<ParDo.BoundMulti<I, O>>() {
-      @Override
-      public void evaluate(ParDo.BoundMulti<I, O> transform, EvaluationContext context) {
-        TupleTag<O> mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform);
-        MultiDoFnFunction<I, O> multifn = new MultiDoFnFunction<>(
-            transform.getFn(),
-            context.getRuntimeContext(),
-            mainOutputTag,
-            getSideInputs(transform.getSideInputs(), context));
-
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-        JavaPairRDD<TupleTag<?>, WindowedValue<?>> all = inRDD
-            .mapPartitionsToPair(multifn)
-            .cache();
-
-        PCollectionTuple pct = context.getOutput(transform);
-        for (Map.Entry<TupleTag<?>, PCollection<?>> e : pct.getAll().entrySet()) {
-          @SuppressWarnings("unchecked")
-          JavaPairRDD<TupleTag<?>, WindowedValue<?>> filtered =
-              all.filter(new TupleTagFilter(e.getKey()));
-          @SuppressWarnings("unchecked")
-          // Object is the best we can do since different outputs can have different tags
-          JavaRDD<WindowedValue<Object>> values =
-              (JavaRDD<WindowedValue<Object>>) (JavaRDD<?>) filtered.values();
-          context.setRDD(e.getValue(), values);
-        }
-      }
-    };
-  }
-
-
-  private static <T> TransformEvaluator<TextIO.Read.Bound<T>> readText() {
-    return new TransformEvaluator<TextIO.Read.Bound<T>>() {
-      @Override
-      public void evaluate(TextIO.Read.Bound<T> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaRDD<WindowedValue<String>> rdd = context.getSparkContext().textFile(pattern)
-                .map(WindowingHelpers.<String>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<TextIO.Write.Bound<T>> writeText() {
-    return new TransformEvaluator<TextIO.Write.Bound<T>>() {
-      @Override
-      public void evaluate(TextIO.Write.Bound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<T, Void> last =
-            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
-            .map(WindowingHelpers.<T>unwindowFunction())
-            .mapToPair(new PairFunction<T, T,
-                    Void>() {
-              @Override
-              public Tuple2<T, Void> call(T t) throws Exception {
-                return new Tuple2<>(t, null);
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-                transform.getShardTemplate(), transform.getFilenamePrefix(),
-                transform.getFilenameSuffix());
-        writeHadoopFile(last, new Configuration(), shardTemplateInfo, Text.class,
-            NullWritable.class, TemplatedTextOutputFormat.class);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<AvroIO.Read.Bound<T>> readAvro() {
-    return new TransformEvaluator<AvroIO.Read.Bound<T>>() {
-      @Override
-      public void evaluate(AvroIO.Read.Bound<T> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaSparkContext jsc = context.getSparkContext();
-        @SuppressWarnings("unchecked")
-        JavaRDD<AvroKey<T>> avroFile = (JavaRDD<AvroKey<T>>) (JavaRDD<?>)
-            jsc.newAPIHadoopFile(pattern,
-                                 AvroKeyInputFormat.class,
-                                 AvroKey.class, NullWritable.class,
-                                 new Configuration()).keys();
-        JavaRDD<WindowedValue<T>> rdd = avroFile.map(
-            new Function<AvroKey<T>, T>() {
-              @Override
-              public T call(AvroKey<T> key) {
-                return key.datum();
-              }
-            }).map(WindowingHelpers.<T>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<AvroIO.Write.Bound<T>> writeAvro() {
-    return new TransformEvaluator<AvroIO.Write.Bound<T>>() {
-      @Override
-      public void evaluate(AvroIO.Write.Bound<T> transform, EvaluationContext context) {
-        Job job;
-        try {
-          job = Job.getInstance();
-        } catch (IOException e) {
-          throw new IllegalStateException(e);
-        }
-        AvroJob.setOutputKeySchema(job, transform.getSchema());
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<AvroKey<T>, NullWritable> last =
-            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
-            .map(WindowingHelpers.<T>unwindowFunction())
-            .mapToPair(new PairFunction<T, AvroKey<T>, NullWritable>() {
-              @Override
-              public Tuple2<AvroKey<T>, NullWritable> call(T t) throws Exception {
-                return new Tuple2<>(new AvroKey<>(t), NullWritable.get());
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-            transform.getShardTemplate(), transform.getFilenamePrefix(),
-            transform.getFilenameSuffix());
-        writeHadoopFile(last, job.getConfiguration(), shardTemplateInfo,
-            AvroKey.class, NullWritable.class, TemplatedAvroKeyOutputFormat.class);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<HadoopIO.Read.Bound<K, V>> readHadoop() {
-    return new TransformEvaluator<HadoopIO.Read.Bound<K, V>>() {
-      @Override
-      public void evaluate(HadoopIO.Read.Bound<K, V> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaSparkContext jsc = context.getSparkContext();
-        @SuppressWarnings ("unchecked")
-        JavaPairRDD<K, V> file = jsc.newAPIHadoopFile(pattern,
-            transform.getFormatClass(),
-            transform.getKeyClass(), transform.getValueClass(),
-            new Configuration());
-        JavaRDD<WindowedValue<KV<K, V>>> rdd =
-            file.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-          @Override
-          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
-            return KV.of(t2._1(), t2._2());
-          }
-        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<HadoopIO.Write.Bound<K, V>> writeHadoop() {
-    return new TransformEvaluator<HadoopIO.Write.Bound<K, V>>() {
-      @Override
-      public void evaluate(HadoopIO.Write.Bound<K, V> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<K, V> last = ((JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context
-            .getInputRDD(transform))
-            .map(WindowingHelpers.<KV<K, V>>unwindowFunction())
-            .mapToPair(new PairFunction<KV<K, V>, K, V>() {
-              @Override
-              public Tuple2<K, V> call(KV<K, V> t) throws Exception {
-                return new Tuple2<>(t.getKey(), t.getValue());
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-                transform.getShardTemplate(), transform.getFilenamePrefix(),
-                transform.getFilenameSuffix());
-        Configuration conf = new Configuration();
-        for (Map.Entry<String, String> e : transform.getConfigurationProperties().entrySet()) {
-          conf.set(e.getKey(), e.getValue());
-        }
-        writeHadoopFile(last, conf, shardTemplateInfo,
-            transform.getKeyClass(), transform.getValueClass(), transform.getFormatClass());
-      }
-    };
-  }
-
-  private static final class ShardTemplateInformation {
-    private final int numShards;
-    private final String shardTemplate;
-    private final String filenamePrefix;
-    private final String filenameSuffix;
-
-    private ShardTemplateInformation(int numShards, String shardTemplate, String
-        filenamePrefix, String filenameSuffix) {
-      this.numShards = numShards;
-      this.shardTemplate = shardTemplate;
-      this.filenamePrefix = filenamePrefix;
-      this.filenameSuffix = filenameSuffix;
-    }
-
-    int getNumShards() {
-      return numShards;
-    }
-
-    String getShardTemplate() {
-      return shardTemplate;
-    }
-
-    String getFilenamePrefix() {
-      return filenamePrefix;
-    }
-
-    String getFilenameSuffix() {
-      return filenameSuffix;
-    }
-  }
-
-  private static <K, V> void writeHadoopFile(JavaPairRDD<K, V> rdd, Configuration conf,
-      ShardTemplateInformation shardTemplateInfo, Class<?> keyClass, Class<?> valueClass,
-      Class<? extends FileOutputFormat> formatClass) {
-    int numShards = shardTemplateInfo.getNumShards();
-    String shardTemplate = shardTemplateInfo.getShardTemplate();
-    String filenamePrefix = shardTemplateInfo.getFilenamePrefix();
-    String filenameSuffix = shardTemplateInfo.getFilenameSuffix();
-    if (numShards != 0) {
-      // number of shards was set explicitly, so repartition
-      rdd = rdd.repartition(numShards);
-    }
-    int actualNumShards = rdd.partitions().size();
-    String template = replaceShardCount(shardTemplate, actualNumShards);
-    String outputDir = getOutputDirectory(filenamePrefix, template);
-    String filePrefix = getOutputFilePrefix(filenamePrefix, template);
-    String fileTemplate = getOutputFileTemplate(filenamePrefix, template);
-
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_PREFIX, filePrefix);
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_TEMPLATE, fileTemplate);
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_SUFFIX, filenameSuffix);
-    rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
-  }
-
-  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
-
-  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
-    return new TransformEvaluator<Window.Bound<T>>() {
-      @Override
-      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<T>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform);
-        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
-        if (windowFn instanceof GlobalWindows) {
-          context.setOutputRDD(transform, inRDD);
-        } else {
-          @SuppressWarnings("unchecked")
-          DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
-          DoFnFunction<T, T> dofn =
-                  new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null);
-          context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
-        }
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Create.Values<T>> create() {
-    return new TransformEvaluator<Create.Values<T>>() {
-      @Override
-      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
-        Iterable<T> elems = transform.getElements();
-        // Use a coder to convert the objects in the PCollection to byte arrays, so they
-        // can be transferred over the network.
-        Coder<T> coder = context.getOutput(transform).getCoder();
-        context.setOutputRDDFromValues(transform, elems, coder);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<View.AsSingleton<T>> viewAsSingleton() {
-    return new TransformEvaluator<View.AsSingleton<T>>() {
-      @Override
-      public void evaluate(View.AsSingleton<T> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-                context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<View.AsIterable<T>> viewAsIter() {
-    return new TransformEvaluator<View.AsIterable<T>>() {
-      @Override
-      public void evaluate(View.AsIterable<T> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-                context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static <R, W> TransformEvaluator<View.CreatePCollectionView<R, W>> createPCollView() {
-    return new TransformEvaluator<View.CreatePCollectionView<R, W>>() {
-      @Override
-      public void evaluate(View.CreatePCollectionView<R, W> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-            context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static final class TupleTagFilter<V>
-      implements Function<Tuple2<TupleTag<V>, WindowedValue<?>>, Boolean> {
-
-    private final TupleTag<V> tag;
-
-    private TupleTagFilter(TupleTag<V> tag) {
-      this.tag = tag;
-    }
-
-    @Override
-    public Boolean call(Tuple2<TupleTag<V>, WindowedValue<?>> input) {
-      return tag.equals(input._1());
-    }
-  }
-
-  private static Map<TupleTag<?>, BroadcastHelper<?>> getSideInputs(
-      List<PCollectionView<?>> views,
-      EvaluationContext context) {
-    if (views == null) {
-      return ImmutableMap.of();
-    } else {
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs = Maps.newHashMap();
-      for (PCollectionView<?> view : views) {
-        Iterable<? extends WindowedValue<?>> collectionView = context.getPCollectionView(view);
-        Coder<Iterable<WindowedValue<?>>> coderInternal = view.getCoderInternal();
-        @SuppressWarnings("unchecked")
-        BroadcastHelper<?> helper =
-            BroadcastHelper.create((Iterable<WindowedValue<?>>) collectionView, coderInternal);
-        //broadcast side inputs
-        helper.broadcast(context.getSparkContext());
-        sideInputs.put(view.getTagInternal(), helper);
-      }
-      return sideInputs;
-    }
-  }
-
-  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
-      .newHashMap();
-
-  static {
-    EVALUATORS.put(TextIO.Read.Bound.class, readText());
-    EVALUATORS.put(TextIO.Write.Bound.class, writeText());
-    EVALUATORS.put(AvroIO.Read.Bound.class, readAvro());
-    EVALUATORS.put(AvroIO.Write.Bound.class, writeAvro());
-    EVALUATORS.put(HadoopIO.Read.Bound.class, readHadoop());
-    EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop());
-    EVALUATORS.put(ParDo.Bound.class, parDo());
-    EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
-    EVALUATORS.put(GroupByKey.GroupByKeyOnly.class, gbk());
-    EVALUATORS.put(Combine.GroupedValues.class, grouped());
-    EVALUATORS.put(Combine.Globally.class, combineGlobally());
-    EVALUATORS.put(Combine.PerKey.class, combinePerKey());
-    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
-    EVALUATORS.put(Create.Values.class, create());
-    EVALUATORS.put(View.AsSingleton.class, viewAsSingleton());
-    EVALUATORS.put(View.AsIterable.class, viewAsIter());
-    EVALUATORS.put(View.CreatePCollectionView.class, createPCollView());
-    EVALUATORS.put(Window.Bound.class, window());
-  }
-
-  public static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
-  getTransformEvaluator(Class<PT> clazz) {
-    @SuppressWarnings("unchecked")
-    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
-    if (transform == null) {
-      throw new IllegalStateException("No TransformEvaluator registered for " + clazz);
-    }
-    return transform;
-  }
-
-  /**
-   * Translator matches Dataflow transformation with the appropriate evaluator.
-   */
-  public static class Translator implements SparkPipelineTranslator {
-
-    @Override
-    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
-      return EVALUATORS.containsKey(clazz);
-    }
-
-    @Override
-    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
-      return getTransformEvaluator(clazz);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
deleted file mode 100644
index bc09d5b..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import org.apache.spark.api.java.function.Function;
-
-/**
- * Helper functions for working with windows.
- */
-public final class WindowingHelpers {
-  private WindowingHelpers() {
-  }
-
-  /**
-   * A function for converting a value to a {@link WindowedValue}. The resulting
-   * {@link WindowedValue} will be in no windows, and will have the default timestamp
-   * and pane.
-   *
-   * @param <T>   The type of the object.
-   * @return A function that accepts an object and returns its {@link WindowedValue}.
-   */
-  public static <T> Function<T, WindowedValue<T>> windowFunction() {
-    return new Function<T, WindowedValue<T>>() {
-      @Override
-      public WindowedValue<T> call(T t) {
-        return WindowedValue.valueInEmptyWindows(t);
-      }
-    };
-  }
-
-  /**
-   * A function for extracting the value from a {@link WindowedValue}.
-   *
-   * @param <T>   The type of the object.
-   * @return A function that accepts a {@link WindowedValue} and returns its value.
-   */
-  public static <T> Function<WindowedValue<T>, T> unwindowFunction() {
-    return new Function<WindowedValue<T>, T>() {
-      @Override
-      public T call(WindowedValue<T> t) {
-        return t.getValue();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
index c3718f4..6c397a2 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
@@ -30,7 +30,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder;
 import com.google.cloud.dataflow.sdk.transforms.Combine;
 import com.google.common.collect.ImmutableList;
 
-import org.apache.beam.runners.spark.SparkRuntimeContext;
+import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
 
 /**
  * This class wraps a map of named aggregators. Spark expects that all accumulators be declared

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
deleted file mode 100644
index f96f4dd..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-
-import org.apache.beam.runners.spark.SparkPipelineOptions;
-
-/**
- * Options used to configure Spark streaming.
- */
-public interface SparkStreamingPipelineOptions extends SparkPipelineOptions {
-  @Description("Timeout to wait (in msec) for the streaming execution so stop, -1 runs until " +
-          "execution is stopped")
-  @Default.Long(-1)
-  Long getTimeout();
-
-  void setTimeout(Long batchInterval);
-
-  @Override
-  @Default.Boolean(true)
-  boolean isStreaming();
-
-  @Override
-  @Default.String("spark streaming dataflow pipeline job")
-  String getAppName();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
deleted file mode 100644
index ae04ebe..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-
-public final class SparkStreamingPipelineOptionsFactory {
-
-  private SparkStreamingPipelineOptionsFactory() {
-  }
-
-  public static SparkStreamingPipelineOptions create() {
-    return PipelineOptionsFactory.as(SparkStreamingPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
deleted file mode 100644
index 256820c..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkStreamingPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
-
-  @Override
-  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkStreamingPipelineOptions
-            .class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
deleted file mode 100644
index 50c889c..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import org.apache.beam.runners.spark.EvaluationContext;
-import org.apache.beam.runners.spark.SparkRuntimeContext;
-
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-
-
-/**
- * Streaming evaluation context helps to handle streaming.
- */
-public class StreamingEvaluationContext extends EvaluationContext {
-
-  private final JavaStreamingContext jssc;
-  private final long timeout;
-  private final Map<PValue, DStreamHolder<?>> pstreams = new LinkedHashMap<>();
-  private final Set<DStreamHolder<?>> leafStreams = new LinkedHashSet<>();
-
-  public StreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
-      JavaStreamingContext jssc, long timeout) {
-    super(jsc, pipeline);
-    this.jssc = jssc;
-    this.timeout = timeout;
-  }
-
-  /**
-   * DStream holder Can also crate a DStream from a supplied queue of values, but mainly for
-   * testing.
-   */
-  private class DStreamHolder<T> {
-
-    private Iterable<Iterable<T>> values;
-    private Coder<T> coder;
-    private JavaDStream<WindowedValue<T>> dStream;
-
-    DStreamHolder(Iterable<Iterable<T>> values, Coder<T> coder) {
-      this.values = values;
-      this.coder = coder;
-    }
-
-    DStreamHolder(JavaDStream<WindowedValue<T>> dStream) {
-      this.dStream = dStream;
-    }
-
-    @SuppressWarnings("unchecked")
-    JavaDStream<WindowedValue<T>> getDStream() {
-      if (dStream == null) {
-        // create the DStream from values
-        Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
-        for (Iterable<T> v : values) {
-          setOutputRDDFromValues(currentTransform.getTransform(), v, coder);
-          rddQueue.offer((JavaRDD<WindowedValue<T>>) getOutputRDD(currentTransform.getTransform()));
-        }
-        // 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);
-      }
-      return dStream;
-    }
-  }
-
-  <T> void setDStreamFromQueue(
-      PTransform<?, ?> transform, Iterable<Iterable<T>> values, Coder<T> coder) {
-    pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder));
-  }
-
-  <T> void setStream(PTransform<?, ?> transform, JavaDStream<WindowedValue<T>> dStream) {
-    PValue pvalue = (PValue) getOutput(transform);
-    DStreamHolder<T> dStreamHolder = new DStreamHolder<>(dStream);
-    pstreams.put(pvalue, dStreamHolder);
-    leafStreams.add(dStreamHolder);
-  }
-
-  boolean hasStream(PTransform<?, ?> transform) {
-    PValue pvalue = (PValue) getInput(transform);
-    return pstreams.containsKey(pvalue);
-  }
-
-  JavaDStreamLike<?, ?, ?> getStream(PTransform<?, ?> transform) {
-    return getStream((PValue) getInput(transform));
-  }
-
-  JavaDStreamLike<?, ?, ?> getStream(PValue pvalue) {
-    DStreamHolder<?> dStreamHolder = pstreams.get(pvalue);
-    JavaDStreamLike<?, ?, ?> dStream = dStreamHolder.getDStream();
-    leafStreams.remove(dStreamHolder);
-    return dStream;
-  }
-
-  // used to set the RDD from the DStream in the RDDHolder for transformation
-  <T> void setInputRDD(
-      PTransform<? extends PInput, ?> transform, JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    setRDD((PValue) getInput(transform), rdd);
-  }
-
-  // used to get the RDD transformation output and use it as the DStream transformation output
-  JavaRDDLike<?, ?> getOutputRDD(PTransform<?, ?> transform) {
-    return getRDD((PValue) getOutput(transform));
-  }
-
-  public JavaStreamingContext getStreamingContext() {
-    return jssc;
-  }
-
-  @Override
-  protected void computeOutputs() {
-    for (DStreamHolder<?> streamHolder : leafStreams) {
-      computeOutput(streamHolder);
-    }
-  }
-
-  private static <T> void computeOutput(DStreamHolder<T> streamHolder) {
-    streamHolder.getDStream().foreachRDD(new Function<JavaRDD<WindowedValue<T>>, Void>() {
-      @Override
-      public Void call(JavaRDD<WindowedValue<T>> rdd) throws Exception {
-        rdd.rdd().cache();
-        rdd.count();
-        return null;
-      }
-    }); // force a DStream action
-  }
-
-  @Override
-  public void close() {
-    if (timeout > 0) {
-      jssc.awaitTerminationOrTimeout(timeout);
-    } else {
-      jssc.awaitTermination();
-    }
-    //TODO: stop gracefully ?
-    jssc.stop(false, false);
-    state = State.DONE;
-    super.close();
-  }
-
-  private State state = State.RUNNING;
-
-  @Override
-  public State getState() {
-    return state;
-  }
-
-  //---------------- override in order to expose in package
-  @Override
-  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
-    return super.getInput(transform);
-  }
-  @Override
-  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
-    return super.getOutput(transform);
-  }
-
-  @Override
-  protected JavaSparkContext getSparkContext() {
-    return super.getSparkContext();
-  }
-
-  @Override
-  protected SparkRuntimeContext getRuntimeContext() {
-    return super.getRuntimeContext();
-  }
-
-  @Override
-  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
-    super.setCurrentTransform(transform);
-  }
-
-  @Override
-  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
-    return super.getCurrentTransform();
-  }
-
-  @Override
-  protected <T> void setOutputRDD(PTransform<?, ?> transform,
-      JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    super.setOutputRDD(transform, rdd);
-  }
-
-  @Override
-  protected <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
-      Coder<T> coder) {
-    super.setOutputRDDFromValues(transform, values, coder);
-  }
-
-  @Override
-  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
-    return super.hasOutputRDD(transform);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
deleted file mode 100644
index 56d0dd9..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
+++ /dev/null
@@ -1,418 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.api.client.util.Lists;
-import com.google.api.client.util.Maps;
-import com.google.api.client.util.Sets;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-import com.google.common.reflect.TypeToken;
-import kafka.serializer.Decoder;
-
-import org.apache.beam.runners.spark.DoFnFunction;
-import org.apache.beam.runners.spark.EvaluationContext;
-import org.apache.beam.runners.spark.SparkPipelineTranslator;
-import org.apache.beam.runners.spark.TransformEvaluator;
-import org.apache.beam.runners.spark.TransformTranslator;
-import org.apache.beam.runners.spark.WindowingHelpers;
-import org.apache.beam.runners.spark.io.ConsoleIO;
-import org.apache.beam.runners.spark.io.CreateStream;
-import org.apache.beam.runners.spark.io.KafkaIO;
-import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
-
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.Durations;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
-import org.apache.spark.streaming.api.java.JavaPairInputDStream;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.kafka.KafkaUtils;
-
-import scala.Tuple2;
-
-
-/**
- * Supports translation between a DataFlow transform, and Spark's operations on DStreams.
- */
-public final class StreamingTransformTranslator {
-
-  private StreamingTransformTranslator() {
-  }
-
-  private static <T> TransformEvaluator<ConsoleIO.Write.Unbound<T>> print() {
-    return new TransformEvaluator<ConsoleIO.Write.Unbound<T>>() {
-      @Override
-      public void evaluate(ConsoleIO.Write.Unbound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
-            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
-            ((StreamingEvaluationContext) context).getStream(transform);
-        dstream.map(WindowingHelpers.<T>unwindowFunction()).print(transform.getNum());
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<KafkaIO.Read.Unbound<K, V>> kafka() {
-    return new TransformEvaluator<KafkaIO.Read.Unbound<K, V>>() {
-      @Override
-      public void evaluate(KafkaIO.Read.Unbound<K, V> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        JavaStreamingContext jssc = sec.getStreamingContext();
-        Class<K> keyClazz = transform.getKeyClass();
-        Class<V> valueClazz = transform.getValueClass();
-        Class<? extends Decoder<K>> keyDecoderClazz = transform.getKeyDecoderClass();
-        Class<? extends Decoder<V>> valueDecoderClazz = transform.getValueDecoderClass();
-        Map<String, String> kafkaParams = transform.getKafkaParams();
-        Set<String> topics = transform.getTopics();
-        JavaPairInputDStream<K, V> inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz,
-                valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics);
-        JavaDStream<WindowedValue<KV<K, V>>> inputStream =
-            inputPairStream.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-          @Override
-          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
-            return KV.of(t2._1(), t2._2());
-          }
-        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
-        sec.setStream(transform, inputStream);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Create.Values<T>> create() {
-    return new TransformEvaluator<Create.Values<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        Iterable<T> elems = transform.getElements();
-        Coder<T> coder = sec.getOutput(transform).getCoder();
-        if (coder != VoidCoder.of()) {
-          // actual create
-          sec.setOutputRDDFromValues(transform, elems, coder);
-        } else {
-          // fake create as an input
-          // creates a stream with a single batch containing a single null element
-          // to invoke following transformations once
-          // to support DataflowAssert
-          sec.setDStreamFromQueue(transform,
-              Collections.<Iterable<Void>>singletonList(Collections.singletonList((Void) null)),
-              (Coder<Void>) coder);
-        }
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<CreateStream.QueuedValues<T>> createFromQueue() {
-    return new TransformEvaluator<CreateStream.QueuedValues<T>>() {
-      @Override
-      public void evaluate(CreateStream.QueuedValues<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        Iterable<Iterable<T>> values = transform.getQueuedValues();
-        Coder<T> coder = sec.getOutput(transform).getCoder();
-        sec.setDStreamFromQueue(transform, values, coder);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
-    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        PCollectionList<T> pcs = sec.getInput(transform);
-        JavaDStream<WindowedValue<T>> first =
-            (JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(0));
-        List<JavaDStream<WindowedValue<T>>> rest = Lists.newArrayListWithCapacity(pcs.size() - 1);
-        for (int i = 1; i < pcs.size(); i++) {
-          rest.add((JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(i)));
-        }
-        JavaDStream<WindowedValue<T>> dstream = sec.getStreamingContext().union(first, rest);
-        sec.setStream(transform, dstream);
-      }
-    };
-  }
-
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> rddTransform(
-      final SparkPipelineTranslator rddTranslator) {
-    return new TransformEvaluator<PT>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(PT transform, EvaluationContext context) {
-        TransformEvaluator<PT> rddEvaluator =
-            rddTranslator.translate((Class<PT>) transform.getClass());
-
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        if (sec.hasStream(transform)) {
-          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
-              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
-              sec.getStream(transform);
-
-          sec.setStream(transform, dStream
-              .transform(new RDDTransform<>(sec, rddEvaluator, transform)));
-        } else {
-          // if the transformation requires direct access to RDD (not in stream)
-          // this is used for "fake" transformations like with DataflowAssert
-          rddEvaluator.evaluate(transform, context);
-        }
-      }
-    };
-  }
-
-  /**
-   * RDD transform function If the transformation function doesn't have an input, create a fake one
-   * as an empty RDD.
-   *
-   * @param <PT> PTransform type
-   */
-  private static final class RDDTransform<PT extends PTransform<?, ?>>
-      implements Function<JavaRDD<WindowedValue<Object>>, JavaRDD<WindowedValue<Object>>> {
-
-    private final StreamingEvaluationContext context;
-    private final AppliedPTransform<?, ?, ?> appliedPTransform;
-    private final TransformEvaluator<PT> rddEvaluator;
-    private final PT transform;
-
-
-    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator<PT> rddEvaluator,
-        PT transform) {
-      this.context = context;
-      this.appliedPTransform = context.getCurrentTransform();
-      this.rddEvaluator = rddEvaluator;
-      this.transform = transform;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public JavaRDD<WindowedValue<Object>>
-        call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
-      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
-      context.setCurrentTransform(appliedPTransform);
-      context.setInputRDD(transform, rdd);
-      rddEvaluator.evaluate(transform, context);
-      if (!context.hasOutputRDD(transform)) {
-        // fake RDD as output
-        context.setOutputRDD(transform,
-            context.getSparkContext().<WindowedValue<Object>>emptyRDD());
-      }
-      JavaRDD<WindowedValue<Object>> outRDD =
-          (JavaRDD<WindowedValue<Object>>) context.getOutputRDD(transform);
-      context.setCurrentTransform(existingAPT);
-      return outRDD;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> foreachRDD(
-      final SparkPipelineTranslator rddTranslator) {
-    return new TransformEvaluator<PT>() {
-      @Override
-      public void evaluate(PT transform, EvaluationContext context) {
-        TransformEvaluator<PT> rddEvaluator =
-            rddTranslator.translate((Class<PT>) transform.getClass());
-
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        if (sec.hasStream(transform)) {
-          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
-              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
-              sec.getStream(transform);
-
-          dStream.foreachRDD(new RDDOutputOperator<>(sec, rddEvaluator, transform));
-        } else {
-          rddEvaluator.evaluate(transform, context);
-        }
-      }
-    };
-  }
-
-  /**
-   * RDD output function.
-   *
-   * @param <PT> PTransform type
-   */
-  private static final class RDDOutputOperator<PT extends PTransform<?, ?>>
-      implements Function<JavaRDD<WindowedValue<Object>>, Void> {
-
-    private final StreamingEvaluationContext context;
-    private final AppliedPTransform<?, ?, ?> appliedPTransform;
-    private final TransformEvaluator<PT> rddEvaluator;
-    private final PT transform;
-
-
-    private RDDOutputOperator(StreamingEvaluationContext context,
-        TransformEvaluator<PT> rddEvaluator, PT transform) {
-      this.context = context;
-      this.appliedPTransform = context.getCurrentTransform();
-      this.rddEvaluator = rddEvaluator;
-      this.transform = transform;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public Void call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
-      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
-      context.setCurrentTransform(appliedPTransform);
-      context.setInputRDD(transform, rdd);
-      rddEvaluator.evaluate(transform, context);
-      context.setCurrentTransform(existingAPT);
-      return null;
-    }
-  }
-
-  private static final TransformTranslator.FieldGetter WINDOW_FG =
-      new TransformTranslator.FieldGetter(Window.Bound.class);
-
-  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
-    return new TransformEvaluator<Window.Bound<T>>() {
-      @Override
-      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        //--- first we apply windowing to the stream
-        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
-        @SuppressWarnings("unchecked")
-        JavaDStream<WindowedValue<T>> dStream =
-            (JavaDStream<WindowedValue<T>>) sec.getStream(transform);
-        if (windowFn instanceof FixedWindows) {
-          Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize()
-              .getMillis());
-          sec.setStream(transform, dStream.window(windowDuration));
-        } else if (windowFn instanceof SlidingWindows) {
-          Duration windowDuration = Durations.milliseconds(((SlidingWindows) windowFn).getSize()
-              .getMillis());
-          Duration slideDuration = Durations.milliseconds(((SlidingWindows) windowFn).getPeriod()
-              .getMillis());
-          sec.setStream(transform, dStream.window(windowDuration, slideDuration));
-        }
-        //--- then we apply windowing to the elements
-        DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
-        DoFnFunction<T, T> dofn = new DoFnFunction<>(addWindowsDoFn,
-            ((StreamingEvaluationContext)context).getRuntimeContext(), null);
-        @SuppressWarnings("unchecked")
-        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
-            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
-            sec.getStream(transform);
-        sec.setStream(transform, dstream.mapPartitions(dofn));
-      }
-    };
-  }
-
-  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
-      .newHashMap();
-
-  static {
-    EVALUATORS.put(ConsoleIO.Write.Unbound.class, print());
-    EVALUATORS.put(CreateStream.QueuedValues.class, createFromQueue());
-    EVALUATORS.put(Create.Values.class, create());
-    EVALUATORS.put(KafkaIO.Read.Unbound.class, kafka());
-    EVALUATORS.put(Window.Bound.class, window());
-    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
-  }
-
-  private static final Set<Class<? extends PTransform>> UNSUPPORTED_EVALUATORS = Sets
-      .newHashSet();
-
-  static {
-    //TODO - add support for the following
-    UNSUPPORTED_EVALUATORS.add(TextIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(TextIO.Write.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(AvroIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(AvroIO.Write.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(HadoopIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(HadoopIO.Write.Bound.class);
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
-      getTransformEvaluator(Class<PT> clazz, SparkPipelineTranslator rddTranslator) {
-    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
-    if (transform == null) {
-      if (UNSUPPORTED_EVALUATORS.contains(clazz)) {
-        throw new UnsupportedOperationException("Dataflow transformation " + clazz
-          .getCanonicalName()
-          + " is currently unsupported by the Spark streaming pipeline");
-      }
-      // DStream transformations will transform an RDD into another RDD
-      // Actions will create output
-      // In Dataflow it depends on the PTransform's Input and Output class
-      Class<?> pTOutputClazz = getPTransformOutputClazz(clazz);
-      if (PDone.class.equals(pTOutputClazz)) {
-        return foreachRDD(rddTranslator);
-      } else {
-        return rddTransform(rddTranslator);
-      }
-    }
-    return transform;
-  }
-
-  private static <PT extends PTransform<?, ?>> Class<?> getPTransformOutputClazz(Class<PT> clazz) {
-    Type[] types = ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments();
-    return TypeToken.of(clazz).resolveType(types[1]).getRawType();
-  }
-
-  /**
-   * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator.
-   * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation
-   */
-  public static class Translator implements SparkPipelineTranslator {
-
-    private final SparkPipelineTranslator rddTranslator;
-
-    public Translator(SparkPipelineTranslator rddTranslator) {
-      this.rddTranslator = rddTranslator;
-    }
-
-    @Override
-    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
-      // streaming includes rdd transformations as well
-      return EVALUATORS.containsKey(clazz) || rddTranslator.hasTranslation(clazz);
-    }
-
-    @Override
-    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
-      return getTransformEvaluator(clazz, rddTranslator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
deleted file mode 100644
index 9c58126..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.apache.beam.runners.spark.SparkPipelineTranslator;
-import org.apache.beam.runners.spark.TransformTranslator;
-
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.Durations;
-
-
-/**
- * Pipeline {@link SparkPipelineRunner.Evaluator} to detect windowing.
- */
-public final class StreamingWindowPipelineDetector extends SparkPipelineRunner.Evaluator {
-
-  // Currently, Spark streaming recommends batches no smaller then 500 msec
-  private static final Duration SPARK_MIN_WINDOW = Durations.milliseconds(500);
-
-  private boolean windowing;
-  private Duration batchDuration;
-
-  public StreamingWindowPipelineDetector(SparkPipelineTranslator translator) {
-    super(translator);
-  }
-
-  private static final TransformTranslator.FieldGetter WINDOW_FG =
-      new TransformTranslator.FieldGetter(Window.Bound.class);
-
-  // Use the smallest window (fixed or sliding) as Spark streaming's batch duration
-  @Override
-  protected <PT extends PTransform<? super PInput, POutput>> void
-      doVisitTransform(TransformTreeNode node) {
-    @SuppressWarnings("unchecked")
-    PT transform = (PT) node.getTransform();
-    @SuppressWarnings("unchecked")
-    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
-    if (transformClass.isAssignableFrom(Window.Bound.class)) {
-      WindowFn<?, ?> windowFn = WINDOW_FG.get("windowFn", transform);
-      if (windowFn instanceof FixedWindows) {
-        setBatchDuration(((FixedWindows) windowFn).getSize());
-      } else if (windowFn instanceof SlidingWindows) {
-        if (((SlidingWindows) windowFn).getOffset().getMillis() > 0) {
-          throw new UnsupportedOperationException("Spark does not support window offsets");
-        }
-        // Sliding window size might as well set the batch duration. Applying the transformation
-        // will add the "slide"
-        setBatchDuration(((SlidingWindows) windowFn).getSize());
-      } else if (!(windowFn instanceof GlobalWindows)) {
-        throw new IllegalStateException("Windowing function not supported: " + windowFn);
-      }
-    }
-  }
-
-  private void setBatchDuration(org.joda.time.Duration duration) {
-    Long durationMillis = duration.getMillis();
-    // validate window size
-    if (durationMillis < SPARK_MIN_WINDOW.milliseconds()) {
-      throw new IllegalArgumentException("Windowing of size " + durationMillis +
-          "msec is not supported!");
-    }
-    // choose the smallest duration to be Spark's batch duration, larger ones will be handled
-    // as window functions  over the batched-stream
-    if (!windowing || this.batchDuration.milliseconds() > durationMillis) {
-      this.batchDuration = Durations.milliseconds(durationMillis);
-    }
-    windowing = true;
-  }
-
-  public boolean isWindowing() {
-    return windowing;
-  }
-
-  public Duration getBatchDuration() {
-    return batchDuration;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
new file mode 100644
index 0000000..37bf849
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.spark.api.java.function.FlatMapFunction;
+
+/**
+ * Dataflow's Do functions correspond to Spark's FlatMap functions.
+ *
+ * @param <I> Input element type.
+ * @param <O> Output element type.
+ */
+public class DoFnFunction<I, O> implements FlatMapFunction<Iterator<WindowedValue<I>>,
+    WindowedValue<O>> {
+  private final DoFn<I, O> mFunction;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  /**
+   * @param fn         DoFunction to be wrapped.
+   * @param runtime    Runtime to apply function in.
+   * @param sideInputs Side inputs used in DoFunction.
+   */
+  public DoFnFunction(DoFn<I, O> fn,
+               SparkRuntimeContext runtime,
+               Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    this.mFunction = fn;
+    this.mRuntimeContext = runtime;
+    this.mSideInputs = sideInputs;
+  }
+
+  @Override
+  public Iterable<WindowedValue<O>> call(Iterator<WindowedValue<I>> iter) throws
+      Exception {
+    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
+    ctxt.setup();
+    mFunction.startBundle(ctxt);
+    return ctxt.getOutputIterable(iter, mFunction);
+  }
+
+  private class ProcCtxt extends SparkProcessContext<I, O, WindowedValue<O>> {
+
+    private final List<WindowedValue<O>> outputs = new LinkedList<>();
+
+    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
+        BroadcastHelper<?>> sideInputs) {
+      super(fn, runtimeContext, sideInputs);
+    }
+
+    @Override
+    public synchronized void output(O o) {
+      outputs.add(windowedValue != null ? windowedValue.withValue(o) :
+          WindowedValue.valueInEmptyWindows(o));
+    }
+
+    @Override
+    public synchronized void output(WindowedValue<O> o) {
+      outputs.add(o);
+    }
+
+    @Override
+    protected void clearOutput() {
+      outputs.clear();
+    }
+
+    @Override
+    protected Iterator<WindowedValue<O>> getOutputIterator() {
+      return outputs.iterator();
+    }
+  }
+
+}


[17/23] incubator-beam git commit: [BEAM-11] second iteration of package reorganisation

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
new file mode 100644
index 0000000..a1ddd44
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+
+
+/**
+ * Evaluation context allows us to define how pipeline instructions.
+ */
+public class EvaluationContext implements EvaluationResult {
+  private final JavaSparkContext jsc;
+  private final Pipeline pipeline;
+  private final SparkRuntimeContext runtime;
+  private final Map<PValue, RDDHolder<?>> pcollections = new LinkedHashMap<>();
+  private final Set<RDDHolder<?>> leafRdds = new LinkedHashSet<>();
+  private final Set<PValue> multireads = new LinkedHashSet<>();
+  private final Map<PValue, Object> pobjects = new LinkedHashMap<>();
+  private final Map<PValue, Iterable<? extends WindowedValue<?>>> pview = new LinkedHashMap<>();
+  protected AppliedPTransform<?, ?, ?> currentTransform;
+
+  public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
+    this.jsc = jsc;
+    this.pipeline = pipeline;
+    this.runtime = new SparkRuntimeContext(jsc, pipeline);
+  }
+
+  /**
+   * Holds an RDD or values for deferred conversion to an RDD if needed. PCollections are
+   * sometimes created from a collection of objects (using RDD parallelize) and then
+   * only used to create View objects; in which case they do not need to be
+   * converted to bytes since they are not transferred across the network until they are
+   * broadcast.
+   */
+  private class RDDHolder<T> {
+
+    private Iterable<T> values;
+    private Coder<T> coder;
+    private JavaRDDLike<WindowedValue<T>, ?> rdd;
+
+    RDDHolder(Iterable<T> values, Coder<T> coder) {
+      this.values = values;
+      this.coder = coder;
+    }
+
+    RDDHolder(JavaRDDLike<WindowedValue<T>, ?> rdd) {
+      this.rdd = rdd;
+    }
+
+    JavaRDDLike<WindowedValue<T>, ?> getRDD() {
+      if (rdd == null) {
+        Iterable<WindowedValue<T>> windowedValues = Iterables.transform(values,
+            new Function<T, WindowedValue<T>>() {
+            @Override
+            public WindowedValue<T> apply(T t) {
+             // TODO: this is wrong if T is a TimestampedValue
+              return WindowedValue.valueInEmptyWindows(t);
+            }
+        });
+        WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder =
+            WindowedValue.getValueOnlyCoder(coder);
+        rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
+            .map(CoderHelpers.fromByteFunction(windowCoder));
+      }
+      return rdd;
+    }
+
+    Iterable<T> getValues(PCollection<T> pcollection) {
+      if (values == null) {
+        coder = pcollection.getCoder();
+        JavaRDDLike<byte[], ?> bytesRDD = rdd.map(WindowingHelpers.<T>unwindowFunction())
+            .map(CoderHelpers.toByteFunction(coder));
+        List<byte[]> clientBytes = bytesRDD.collect();
+        values = Iterables.transform(clientBytes, new Function<byte[], T>() {
+          @Override
+          public T apply(byte[] bytes) {
+            return CoderHelpers.fromByteArray(bytes, coder);
+          }
+        });
+      }
+      return values;
+    }
+
+    Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
+      return Iterables.transform(get(pcollection), new Function<T, WindowedValue<T>>() {
+        @Override
+        public WindowedValue<T> apply(T t) {
+          return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place?
+        }
+      });
+    }
+  }
+
+  protected JavaSparkContext getSparkContext() {
+    return jsc;
+  }
+
+  protected Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  protected SparkRuntimeContext getRuntimeContext() {
+    return runtime;
+  }
+
+  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
+    this.currentTransform = transform;
+  }
+
+  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
+    return currentTransform;
+  }
+
+  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
+    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
+        "can only be called with current transform");
+    @SuppressWarnings("unchecked")
+    I input = (I) currentTransform.getInput();
+    return input;
+  }
+
+  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
+    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
+        "can only be called with current transform");
+    @SuppressWarnings("unchecked")
+    O output = (O) currentTransform.getOutput();
+    return output;
+  }
+
+  protected  <T> void setOutputRDD(PTransform<?, ?> transform,
+      JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    setRDD((PValue) getOutput(transform), rdd);
+  }
+
+  protected  <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
+      Coder<T> coder) {
+    pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder));
+  }
+
+  void setPView(PValue view, Iterable<? extends WindowedValue<?>> value) {
+    pview.put(view, value);
+  }
+
+  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
+    PValue pvalue = (PValue) getOutput(transform);
+    return pcollections.containsKey(pvalue);
+  }
+
+  protected JavaRDDLike<?, ?> getRDD(PValue pvalue) {
+    RDDHolder<?> rddHolder = pcollections.get(pvalue);
+    JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
+    leafRdds.remove(rddHolder);
+    if (multireads.contains(pvalue)) {
+      // Ensure the RDD is marked as cached
+      rdd.rdd().cache();
+    } else {
+      multireads.add(pvalue);
+    }
+    return rdd;
+  }
+
+  protected <T> void setRDD(PValue pvalue, JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    try {
+      rdd.rdd().setName(pvalue.getName());
+    } catch (IllegalStateException e) {
+      // name not set, ignore
+    }
+    RDDHolder<T> rddHolder = new RDDHolder<>(rdd);
+    pcollections.put(pvalue, rddHolder);
+    leafRdds.add(rddHolder);
+  }
+
+  JavaRDDLike<?, ?> getInputRDD(PTransform<? extends PInput, ?> transform) {
+    return getRDD((PValue) getInput(transform));
+  }
+
+
+  <T> Iterable<? extends WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
+    return pview.get(view);
+  }
+
+  /**
+   * Computes the outputs for all RDDs that are leaves in the DAG and do not have any
+   * actions (like saving to a file) registered on them (i.e. they are performed for side
+   * effects).
+   */
+  public void computeOutputs() {
+    for (RDDHolder<?> rddHolder : leafRdds) {
+      JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
+      rdd.rdd().cache(); // cache so that any subsequent get() is cheap
+      rdd.count(); // force the RDD to be computed
+    }
+  }
+
+  @Override
+  public <T> T get(PValue value) {
+    if (pobjects.containsKey(value)) {
+      @SuppressWarnings("unchecked")
+      T result = (T) pobjects.get(value);
+      return result;
+    }
+    if (pcollections.containsKey(value)) {
+      JavaRDDLike<?, ?> rdd = pcollections.get(value).getRDD();
+      @SuppressWarnings("unchecked")
+      T res = (T) Iterables.getOnlyElement(rdd.collect());
+      pobjects.put(value, res);
+      return res;
+    }
+    throw new IllegalStateException("Cannot resolve un-known PObject: " + value);
+  }
+
+  @Override
+  public <T> T getAggregatorValue(String named, Class<T> resultType) {
+    return runtime.getAggregatorValue(named, resultType);
+  }
+
+  @Override
+  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
+      throws AggregatorRetrievalException {
+    return runtime.getAggregatorValues(aggregator);
+  }
+
+  @Override
+  public <T> Iterable<T> get(PCollection<T> pcollection) {
+    @SuppressWarnings("unchecked")
+    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
+    return rddHolder.getValues(pcollection);
+  }
+
+  <T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
+    @SuppressWarnings("unchecked")
+    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
+    return rddHolder.getWindowedValues(pcollection);
+  }
+
+  @Override
+  public void close() {
+    SparkContextFactory.stopSparkContext(jsc);
+  }
+
+  /** The runner is blocking. */
+  @Override
+  public State getState() {
+    return State.DONE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
new file mode 100644
index 0000000..cecf962
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.joda.time.Instant;
+import scala.Tuple2;
+
+/**
+ * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enriching the
+ * underlying data with multiple TupleTags.
+ *
+ * @param <I> Input type for DoFunction.
+ * @param <O> Output type for DoFunction.
+ */
+class MultiDoFnFunction<I, O>
+    implements PairFlatMapFunction<Iterator<WindowedValue<I>>, TupleTag<?>, WindowedValue<?>> {
+  private final DoFn<I, O> mFunction;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final TupleTag<O> mMainOutputTag;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  MultiDoFnFunction(
+      DoFn<I, O> fn,
+      SparkRuntimeContext runtimeContext,
+      TupleTag<O> mainOutputTag,
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    this.mFunction = fn;
+    this.mRuntimeContext = runtimeContext;
+    this.mMainOutputTag = mainOutputTag;
+    this.mSideInputs = sideInputs;
+  }
+
+  @Override
+  public Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>>
+      call(Iterator<WindowedValue<I>> iter) throws Exception {
+    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
+    mFunction.startBundle(ctxt);
+    ctxt.setup();
+    return ctxt.getOutputIterable(iter, mFunction);
+  }
+
+  private class ProcCtxt extends SparkProcessContext<I, O, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
+        BroadcastHelper<?>> sideInputs) {
+      super(fn, runtimeContext, sideInputs);
+    }
+
+    @Override
+    public synchronized void output(O o) {
+      outputs.put(mMainOutputTag, windowedValue.withValue(o));
+    }
+
+    @Override
+    public synchronized void output(WindowedValue<O> o) {
+      outputs.put(mMainOutputTag, o);
+    }
+
+    @Override
+    public synchronized <T> void sideOutput(TupleTag<T> tag, T t) {
+      outputs.put(tag, windowedValue.withValue(t));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
+      outputs.put(tupleTag, WindowedValue.of(t, instant,
+          windowedValue.getWindows(), windowedValue.getPane()));
+    }
+
+    @Override
+    protected void clearOutput() {
+      outputs.clear();
+    }
+
+    @Override
+    protected Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> getOutputIterator() {
+      return Iterators.transform(outputs.entries().iterator(),
+          new Function<Map.Entry<TupleTag<?>, WindowedValue<?>>,
+              Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+        @Override
+        public Tuple2<TupleTag<?>, WindowedValue<?>> apply(Map.Entry<TupleTag<?>,
+            WindowedValue<?>> input) {
+          return new Tuple2<TupleTag<?>, WindowedValue<?>>(input.getKey(), input.getValue());
+        }
+      });
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
new file mode 100644
index 0000000..2bc8a7b
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.serializer.KryoSerializer;
+
+public final class SparkContextFactory {
+
+  /**
+   * If the property {@code dataflow.spark.test.reuseSparkContext} is set to
+   * {@code true} then the Spark context will be reused for dataflow pipelines.
+   * This property should only be enabled for tests.
+   */
+  static final String TEST_REUSE_SPARK_CONTEXT =
+      "dataflow.spark.test.reuseSparkContext";
+  private static JavaSparkContext sparkContext;
+  private static String sparkMaster;
+
+  private SparkContextFactory() {
+  }
+
+  public static synchronized JavaSparkContext getSparkContext(String master, String appName) {
+    if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
+      if (sparkContext == null) {
+        sparkContext = createSparkContext(master, appName);
+        sparkMaster = master;
+      } else if (!master.equals(sparkMaster)) {
+        throw new IllegalArgumentException(String.format("Cannot reuse spark context " +
+                "with different spark master URL. Existing: %s, requested: %s.",
+            sparkMaster, master));
+      }
+      return sparkContext;
+    } else {
+      return createSparkContext(master, appName);
+    }
+  }
+
+  static synchronized void stopSparkContext(JavaSparkContext context) {
+    if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
+      context.stop();
+    }
+  }
+
+  private static JavaSparkContext createSparkContext(String master, String appName) {
+    SparkConf conf = new SparkConf();
+    conf.setMaster(master);
+    conf.setAppName(appName);
+    conf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
+    return new JavaSparkContext(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
new file mode 100644
index 0000000..0186c8c
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+
+/**
+ * Pipeline {@link SparkPipelineRunner.Evaluator} for Spark.
+ */
+public final class SparkPipelineEvaluator extends SparkPipelineRunner.Evaluator {
+
+  private final EvaluationContext ctxt;
+
+  public SparkPipelineEvaluator(EvaluationContext ctxt, SparkPipelineTranslator translator) {
+    super(translator);
+    this.ctxt = ctxt;
+  }
+
+  @Override
+  protected <PT extends PTransform<? super PInput, POutput>> void doVisitTransform(TransformTreeNode
+      node) {
+    @SuppressWarnings("unchecked")
+    PT transform = (PT) node.getTransform();
+    @SuppressWarnings("unchecked")
+    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
+    @SuppressWarnings("unchecked") TransformEvaluator<PT> evaluator =
+        (TransformEvaluator<PT>) translator.translate(transformClass);
+    LOG.info("Evaluating {}", transform);
+    AppliedPTransform<PInput, POutput, PT> appliedTransform =
+        AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform);
+    ctxt.setCurrentTransform(appliedTransform);
+    evaluator.evaluate(transform, ctxt);
+    ctxt.setCurrentTransform(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
new file mode 100644
index 0000000..2b6804e
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+
+public final class SparkPipelineOptionsFactory {
+  private SparkPipelineOptionsFactory() {
+  }
+
+  public static SparkPipelineOptions create() {
+    return PipelineOptionsFactory.as(SparkPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
new file mode 100644
index 0000000..9775b3e
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+
+public class SparkPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
+  @Override
+  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
new file mode 100644
index 0000000..e44d999
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+
+public class SparkPipelineRunnerRegistrar implements PipelineRunnerRegistrar {
+  @Override
+  public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+    return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
new file mode 100644
index 0000000..ac1c685
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+/**
+ * Translator to support translation between Dataflow transformations and Spark transformations.
+ */
+public interface SparkPipelineTranslator {
+
+  boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz);
+
+  <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
new file mode 100644
index 0000000..bfcdd80
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.util.state.*;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class SparkProcessContext<I, O, V> extends DoFn<I, O>.ProcessContext {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class);
+
+  private final DoFn<I, O> fn;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  protected WindowedValue<I> windowedValue;
+
+  SparkProcessContext(DoFn<I, O> fn,
+      SparkRuntimeContext runtime,
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    fn.super();
+    this.fn = fn;
+    this.mRuntimeContext = runtime;
+    this.mSideInputs = sideInputs;
+  }
+
+  void setup() {
+    setupDelegateAggregators();
+  }
+
+  @Override
+  public PipelineOptions getPipelineOptions() {
+    return mRuntimeContext.getPipelineOptions();
+  }
+
+  @Override
+  public <T> T sideInput(PCollectionView<T> view) {
+    @SuppressWarnings("unchecked")
+    BroadcastHelper<Iterable<WindowedValue<?>>> broadcastHelper =
+        (BroadcastHelper<Iterable<WindowedValue<?>>>) mSideInputs.get(view.getTagInternal());
+    Iterable<WindowedValue<?>> contents = broadcastHelper.getValue();
+    return view.fromIterableInternal(contents);
+  }
+
+  @Override
+  public abstract void output(O output);
+
+  public abstract void output(WindowedValue<O> output);
+
+  @Override
+  public <T> void sideOutput(TupleTag<T> tupleTag, T t) {
+    String message = "sideOutput is an unsupported operation for doFunctions, use a " +
+        "MultiDoFunction instead.";
+    LOG.warn(message);
+    throw new UnsupportedOperationException(message);
+  }
+
+  @Override
+  public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
+    String message =
+        "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " +
+            "MultiDoFunction instead.";
+    LOG.warn(message);
+    throw new UnsupportedOperationException(message);
+  }
+
+  @Override
+  public <AI, AO> Aggregator<AI, AO> createAggregatorInternal(
+      String named,
+      Combine.CombineFn<AI, ?, AO> combineFn) {
+    return mRuntimeContext.createAggregator(named, combineFn);
+  }
+
+  @Override
+  public I element() {
+    return windowedValue.getValue();
+  }
+
+  @Override
+  public void outputWithTimestamp(O output, Instant timestamp) {
+    output(WindowedValue.of(output, timestamp,
+        windowedValue.getWindows(), windowedValue.getPane()));
+  }
+
+  @Override
+  public Instant timestamp() {
+    return windowedValue.getTimestamp();
+  }
+
+  @Override
+  public BoundedWindow window() {
+    if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+      throw new UnsupportedOperationException(
+          "window() is only available in the context of a DoFn marked as RequiresWindow.");
+    }
+    return Iterables.getOnlyElement(windowedValue.getWindows());
+  }
+
+  @Override
+  public PaneInfo pane() {
+    return windowedValue.getPane();
+  }
+
+  @Override
+  public WindowingInternals<I, O> windowingInternals() {
+    return new WindowingInternals<I, O>() {
+
+      @Override
+      public Collection<? extends BoundedWindow> windows() {
+        return windowedValue.getWindows();
+      }
+
+      @Override
+      public void outputWindowedValue(O output, Instant timestamp, Collection<?
+          extends BoundedWindow> windows, PaneInfo paneInfo) {
+        output(WindowedValue.of(output, timestamp, windows, paneInfo));
+      }
+
+      @Override
+      public StateInternals stateInternals() {
+        //TODO: implement state internals.
+        // This is a temporary placeholder to get the TfIdfTest
+        // working for the initial Beam code drop.
+        return InMemoryStateInternals.forKey("DUMMY");
+      }
+
+      @Override
+      public TimerInternals timerInternals() {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#timerInternals() is not yet supported.");
+      }
+
+      @Override
+      public PaneInfo pane() {
+        return windowedValue.getPane();
+      }
+
+      @Override
+      public <T> void writePCollectionViewData(TupleTag<?> tag,
+          Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#writePCollectionViewData() is not yet supported.");
+      }
+
+      @Override
+      public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#sideInput() is not yet supported.");
+      }
+    };
+  }
+
+  protected abstract void clearOutput();
+  protected abstract Iterator<V> getOutputIterator();
+
+  protected Iterable<V> getOutputIterable(final Iterator<WindowedValue<I>> iter,
+      final DoFn<I, O> doFn) {
+    return new Iterable<V>() {
+      @Override
+      public Iterator<V> iterator() {
+        return new ProcCtxtIterator(iter, doFn);
+      }
+    };
+  }
+
+  private class ProcCtxtIterator extends AbstractIterator<V> {
+
+    private final Iterator<WindowedValue<I>> inputIterator;
+    private final DoFn<I, O> doFn;
+    private Iterator<V> outputIterator;
+    private boolean calledFinish;
+
+    ProcCtxtIterator(Iterator<WindowedValue<I>> iterator, DoFn<I, O> doFn) {
+      this.inputIterator = iterator;
+      this.doFn = doFn;
+      this.outputIterator = getOutputIterator();
+    }
+
+    @Override
+    protected V computeNext() {
+      // Process each element from the (input) iterator, which produces, zero, one or more
+      // output elements (of type V) in the output iterator. Note that the output
+      // collection (and iterator) is reset between each call to processElement, so the
+      // collection only holds the output values for each call to processElement, rather
+      // than for the whole partition (which would use too much memory).
+      while (true) {
+        if (outputIterator.hasNext()) {
+          return outputIterator.next();
+        } else if (inputIterator.hasNext()) {
+          clearOutput();
+          windowedValue = inputIterator.next();
+          try {
+            doFn.processElement(SparkProcessContext.this);
+          } catch (Exception e) {
+            throw new SparkProcessException(e);
+          }
+          outputIterator = getOutputIterator();
+        } else {
+          // no more input to consume, but finishBundle can produce more output
+          if (!calledFinish) {
+            clearOutput();
+            try {
+              calledFinish = true;
+              doFn.finishBundle(SparkProcessContext.this);
+            } catch (Exception e) {
+              throw new SparkProcessException(e);
+            }
+            outputIterator = getOutputIterator();
+            continue; // try to consume outputIterator from start of loop
+          }
+          return endOfData();
+        }
+      }
+    }
+  }
+
+  public static class SparkProcessException extends RuntimeException {
+    SparkProcessException(Throwable t) {
+      super(t);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
new file mode 100644
index 0000000..bf618c4
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Max;
+import com.google.cloud.dataflow.sdk.transforms.Min;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.beam.runners.spark.aggregators.AggAccumParam;
+import org.apache.beam.runners.spark.aggregators.NamedAggregators;
+
+import org.apache.spark.Accumulator;
+import org.apache.spark.api.java.JavaSparkContext;
+
+
+/**
+ * The SparkRuntimeContext allows us to define useful features on the client side before our
+ * data flow program is launched.
+ */
+public class SparkRuntimeContext implements Serializable {
+  /**
+   * An accumulator that is a map from names to aggregators.
+   */
+  private final Accumulator<NamedAggregators> accum;
+
+  private final String serializedPipelineOptions;
+
+  /**
+   * Map fo names to dataflow aggregators.
+   */
+  private final Map<String, Aggregator<?, ?>> aggregators = new HashMap<>();
+  private transient CoderRegistry coderRegistry;
+
+  SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) {
+    this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam());
+    this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions());
+  }
+
+  private static String serializePipelineOptions(PipelineOptions pipelineOptions) {
+    try {
+      return new ObjectMapper().writeValueAsString(pipelineOptions);
+    } catch (JsonProcessingException e) {
+      throw new IllegalStateException("Failed to serialize the pipeline options.", e);
+    }
+  }
+
+  private static PipelineOptions deserializePipelineOptions(String serializedPipelineOptions) {
+    try {
+      return new ObjectMapper().readValue(serializedPipelineOptions, PipelineOptions.class);
+    } catch (IOException e) {
+      throw new IllegalStateException("Failed to deserialize the pipeline options.", e);
+    }
+  }
+
+  /**
+   * Retrieves corresponding value of an aggregator.
+   *
+   * @param aggregatorName Name of the aggregator to retrieve the value of.
+   * @param typeClass      Type class of value to be retrieved.
+   * @param <T>            Type of object to be returned.
+   * @return The value of the aggregator.
+   */
+  public <T> T getAggregatorValue(String aggregatorName, Class<T> typeClass) {
+    return accum.value().getValue(aggregatorName, typeClass);
+  }
+
+  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
+    @SuppressWarnings("unchecked")
+    Class<T> aggValueClass = (Class<T>) aggregator.getCombineFn().getOutputType().getRawType();
+    final T aggregatorValue = getAggregatorValue(aggregator.getName(), aggValueClass);
+    return new AggregatorValues<T>() {
+      @Override
+      public Collection<T> getValues() {
+        return ImmutableList.of(aggregatorValue);
+      }
+
+      @Override
+      public Map<String, T> getValuesAtSteps() {
+        throw new UnsupportedOperationException("getValuesAtSteps is not supported.");
+      }
+    };
+  }
+
+  public synchronized PipelineOptions getPipelineOptions() {
+    return deserializePipelineOptions(serializedPipelineOptions);
+  }
+
+  /**
+   * Creates and aggregator and associates it with the specified name.
+   *
+   * @param named     Name of aggregator.
+   * @param combineFn Combine function used in aggregation.
+   * @param <IN>      Type of inputs to aggregator.
+   * @param <INTER>   Intermediate data type
+   * @param <OUT>     Type of aggregator outputs.
+   * @return Specified aggregator
+   */
+  public synchronized <IN, INTER, OUT> Aggregator<IN, OUT> createAggregator(
+      String named,
+      Combine.CombineFn<? super IN, INTER, OUT> combineFn) {
+    @SuppressWarnings("unchecked")
+    Aggregator<IN, OUT> aggregator = (Aggregator<IN, OUT>) aggregators.get(named);
+    if (aggregator == null) {
+      @SuppressWarnings("unchecked")
+      NamedAggregators.CombineFunctionState<IN, INTER, OUT> state =
+          new NamedAggregators.CombineFunctionState<>(
+              (Combine.CombineFn<IN, INTER, OUT>) combineFn,
+              (Coder<IN>) getCoder(combineFn),
+              this);
+      accum.add(new NamedAggregators(named, state));
+      aggregator = new SparkAggregator<>(named, state);
+      aggregators.put(named, aggregator);
+    }
+    return aggregator;
+  }
+
+  public CoderRegistry getCoderRegistry() {
+    if (coderRegistry == null) {
+      coderRegistry = new CoderRegistry();
+      coderRegistry.registerStandardCoders();
+    }
+    return coderRegistry;
+  }
+
+  private Coder<?> getCoder(Combine.CombineFn<?, ?, ?> combiner) {
+    try {
+      if (combiner.getClass() == Sum.SumIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Sum.SumLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Sum.SumDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else if (combiner.getClass() == Min.MinIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Min.MinLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Min.MinDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else if (combiner.getClass() == Max.MaxIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Max.MaxLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Max.MaxDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else {
+        throw new IllegalArgumentException("unsupported combiner in Aggregator: "
+            + combiner.getClass().getName());
+      }
+    } catch (CannotProvideCoderException e) {
+      throw new IllegalStateException("Could not determine default coder for combiner", e);
+    }
+  }
+
+  /**
+   * Initialize spark aggregators exactly once.
+   *
+   * @param <IN> Type of element fed in to aggregator.
+   */
+  private static class SparkAggregator<IN, OUT> implements Aggregator<IN, OUT>, Serializable {
+    private final String name;
+    private final NamedAggregators.State<IN, ?, OUT> state;
+
+    SparkAggregator(String name, NamedAggregators.State<IN, ?, OUT> state) {
+      this.name = name;
+      this.state = state;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public void addValue(IN elem) {
+      state.update(elem);
+    }
+
+    @Override
+    public Combine.CombineFn<IN, ?, OUT> getCombineFn() {
+      return state.getCombineFn();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
new file mode 100644
index 0000000..d8481bf
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import java.io.Serializable;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+public interface TransformEvaluator<PT extends PTransform<?, ?>> extends Serializable {
+  void evaluate(PT transform, EvaluationContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
new file mode 100644
index 0000000..0bd047c
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
@@ -0,0 +1,808 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.avro.mapred.AvroKey;
+import org.apache.avro.mapreduce.AvroJob;
+import org.apache.avro.mapreduce.AvroKeyInputFormat;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
+import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper;
+import org.apache.beam.runners.spark.io.hadoop.TemplatedAvroKeyOutputFormat;
+import org.apache.beam.runners.spark.io.hadoop.TemplatedTextOutputFormat;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.beam.runners.spark.util.ByteArray;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Tuple2;
+
+/**
+ * Supports translation between a DataFlow transform, and Spark's operations on RDDs.
+ */
+public final class TransformTranslator {
+
+  private TransformTranslator() {
+  }
+
+  public static class FieldGetter {
+    private final Map<String, Field> fields;
+
+    public FieldGetter(Class<?> clazz) {
+      this.fields = Maps.newHashMap();
+      for (Field f : clazz.getDeclaredFields()) {
+        f.setAccessible(true);
+        this.fields.put(f.getName(), f);
+      }
+    }
+
+    public <T> T get(String fieldname, Object value) {
+      try {
+        @SuppressWarnings("unchecked")
+        T fieldValue = (T) fields.get(fieldname).get(value);
+        return fieldValue;
+      } catch (IllegalAccessException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+  }
+
+  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
+    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
+        PCollectionList<T> pcs = context.getInput(transform);
+        JavaRDD<WindowedValue<T>>[] rdds = new JavaRDD[pcs.size()];
+        for (int i = 0; i < rdds.length; i++) {
+          rdds[i] = (JavaRDD<WindowedValue<T>>) context.getRDD(pcs.get(i));
+        }
+        JavaRDD<WindowedValue<T>> rdd = context.getSparkContext().union(rdds);
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>> gbk() {
+    return new TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>>() {
+      @Override
+      public void evaluate(GroupByKey.GroupByKeyOnly<K, V> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, V>>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context.getInputRDD(transform);
+        @SuppressWarnings("unchecked")
+        KvCoder<K, V> coder = (KvCoder<K, V>) context.getInput(transform).getCoder();
+        Coder<K> keyCoder = coder.getKeyCoder();
+        Coder<V> valueCoder = coder.getValueCoder();
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaRDDLike<WindowedValue<KV<K, Iterable<V>>>, ?> outRDD = fromPair(
+              toPair(inRDD.map(WindowingHelpers.<KV<K, V>>unwindowFunction()))
+            .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder))
+            .groupByKey()
+            .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder)))
+            // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
+            .map(WindowingHelpers.<KV<K, Iterable<V>>>windowFunction());
+        context.setOutputRDD(transform, outRDD);
+      }
+    };
+  }
+
+  private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class);
+
+  private static <K, VI, VO> TransformEvaluator<Combine.GroupedValues<K, VI, VO>> grouped() {
+    return new TransformEvaluator<Combine.GroupedValues<K, VI, VO>>() {
+      @Override
+      public void evaluate(Combine.GroupedValues<K, VI, VO> transform, EvaluationContext context) {
+        Combine.KeyedCombineFn<K, VI, ?, VO> keyed = GROUPED_FG.get("fn", transform);
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?>) context.getInputRDD(transform);
+        context.setOutputRDD(transform,
+            inRDD.map(new KVFunction<>(keyed)));
+      }
+    };
+  }
+
+  private static final FieldGetter COMBINE_GLOBALLY_FG = new FieldGetter(Combine.Globally.class);
+
+  private static <I, A, O> TransformEvaluator<Combine.Globally<I, O>> combineGlobally() {
+    return new TransformEvaluator<Combine.Globally<I, O>>() {
+
+      @Override
+      public void evaluate(Combine.Globally<I, O> transform, EvaluationContext context) {
+        final Combine.CombineFn<I, A, O> globally = COMBINE_GLOBALLY_FG.get("fn", transform);
+
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRdd =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+
+        final Coder<I> iCoder = context.getInput(transform).getCoder();
+        final Coder<A> aCoder;
+        try {
+          aCoder = globally.getAccumulatorCoder(
+              context.getPipeline().getCoderRegistry(), iCoder);
+        } catch (CannotProvideCoderException e) {
+          throw new IllegalStateException("Could not determine coder for accumulator", e);
+        }
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaRDD<byte[]> inRddBytes = inRdd
+            .map(WindowingHelpers.<I>unwindowFunction())
+            .map(CoderHelpers.toByteFunction(iCoder));
+
+        /*A*/ byte[] acc = inRddBytes.aggregate(
+            CoderHelpers.toByteArray(globally.createAccumulator(), aCoder),
+            new Function2</*A*/ byte[], /*I*/ byte[], /*A*/ byte[]>() {
+              @Override
+              public /*A*/ byte[] call(/*A*/ byte[] ab, /*I*/ byte[] ib) throws Exception {
+                A a = CoderHelpers.fromByteArray(ab, aCoder);
+                I i = CoderHelpers.fromByteArray(ib, iCoder);
+                return CoderHelpers.toByteArray(globally.addInput(a, i), aCoder);
+              }
+            },
+            new Function2</*A*/ byte[], /*A*/ byte[], /*A*/ byte[]>() {
+              @Override
+              public /*A*/ byte[] call(/*A*/ byte[] a1b, /*A*/ byte[] a2b) throws Exception {
+                A a1 = CoderHelpers.fromByteArray(a1b, aCoder);
+                A a2 = CoderHelpers.fromByteArray(a2b, aCoder);
+                // don't use Guava's ImmutableList.of as values may be null
+                List<A> accumulators = Collections.unmodifiableList(Arrays.asList(a1, a2));
+                A merged = globally.mergeAccumulators(accumulators);
+                return CoderHelpers.toByteArray(merged, aCoder);
+              }
+            }
+        );
+        O output = globally.extractOutput(CoderHelpers.fromByteArray(acc, aCoder));
+
+        Coder<O> coder = context.getOutput(transform).getCoder();
+        JavaRDD<byte[]> outRdd = context.getSparkContext().parallelize(
+            // don't use Guava's ImmutableList.of as output may be null
+            CoderHelpers.toByteArrays(Collections.singleton(output), coder));
+        context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder))
+            .map(WindowingHelpers.<O>windowFunction()));
+      }
+    };
+  }
+
+  private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class);
+
+  private static <K, VI, VA, VO> TransformEvaluator<Combine.PerKey<K, VI, VO>> combinePerKey() {
+    return new TransformEvaluator<Combine.PerKey<K, VI, VO>>() {
+      @Override
+      public void evaluate(Combine.PerKey<K, VI, VO> transform, EvaluationContext context) {
+        final Combine.KeyedCombineFn<K, VI, VA, VO> keyed =
+            COMBINE_PERKEY_FG.get("fn", transform);
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<KV<K, VI>>, ?> inRdd =
+            (JavaRDDLike<WindowedValue<KV<K, VI>>, ?>) context.getInputRDD(transform);
+
+        @SuppressWarnings("unchecked")
+        KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
+        Coder<K> keyCoder = inputCoder.getKeyCoder();
+        Coder<VI> viCoder = inputCoder.getValueCoder();
+        Coder<VA> vaCoder;
+        try {
+          vaCoder = keyed.getAccumulatorCoder(
+              context.getPipeline().getCoderRegistry(), keyCoder, viCoder);
+        } catch (CannotProvideCoderException e) {
+          throw new IllegalStateException("Could not determine coder for accumulator", e);
+        }
+        Coder<KV<K, VI>> kviCoder = KvCoder.of(keyCoder, viCoder);
+        Coder<KV<K, VA>> kvaCoder = KvCoder.of(keyCoder, vaCoder);
+
+        // We need to duplicate K as both the key of the JavaPairRDD as well as inside the value,
+        // since the functions passed to combineByKey don't receive the associated key of each
+        // value, and we need to map back into methods in Combine.KeyedCombineFn, which each
+        // require the key in addition to the VI's and VA's being merged/accumulated. Once Spark
+        // provides a way to include keys in the arguments of combine/merge functions, we won't
+        // need to duplicate the keys anymore.
+
+        // Key has to bw windowed in order to group by window as well
+        JavaPairRDD<WindowedValue<K>, WindowedValue<KV<K, VI>>> inRddDuplicatedKeyPair =
+            inRdd.mapToPair(
+                new PairFunction<WindowedValue<KV<K, VI>>, WindowedValue<K>,
+                    WindowedValue<KV<K, VI>>>() {
+                  @Override
+                  public Tuple2<WindowedValue<K>,
+                      WindowedValue<KV<K, VI>>> call(WindowedValue<KV<K, VI>> kv) {
+                    WindowedValue<K> wk = WindowedValue.of(kv.getValue().getKey(),
+                        kv.getTimestamp(), kv.getWindows(), kv.getPane());
+                    return new Tuple2<>(wk, kv);
+                  }
+                });
+        //-- windowed coders
+        final WindowedValue.FullWindowedValueCoder<K> wkCoder =
+                WindowedValue.FullWindowedValueCoder.of(keyCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+        final WindowedValue.FullWindowedValueCoder<KV<K, VI>> wkviCoder =
+                WindowedValue.FullWindowedValueCoder.of(kviCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+        final WindowedValue.FullWindowedValueCoder<KV<K, VA>> wkvaCoder =
+                WindowedValue.FullWindowedValueCoder.of(kvaCoder,
+                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
+
+        // Use coders to convert objects in the PCollection to byte arrays, so they
+        // can be transferred over the network for the shuffle.
+        JavaPairRDD<ByteArray, byte[]> inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair
+            .mapToPair(CoderHelpers.toByteFunction(wkCoder, wkviCoder));
+
+        // The output of combineByKey will be "VA" (accumulator) types rather than "VO" (final
+        // output types) since Combine.CombineFn only provides ways to merge VAs, and no way
+        // to merge VOs.
+        JavaPairRDD</*K*/ ByteArray, /*KV<K, VA>*/ byte[]> accumulatedBytes =
+            inRddDuplicatedKeyPairBytes.combineByKey(
+            new Function</*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VI>*/ byte[] input) {
+                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
+                VA va = keyed.createAccumulator(wkvi.getValue().getKey());
+                va = keyed.addInput(wkvi.getValue().getKey(), va, wkvi.getValue().getValue());
+                WindowedValue<KV<K, VA>> wkva =
+                    WindowedValue.of(KV.of(wkvi.getValue().getKey(), va), wkvi.getTimestamp(),
+                    wkvi.getWindows(), wkvi.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            },
+            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc,
+                  /*KV<K, VI>*/ byte[] input) {
+                WindowedValue<KV<K, VA>> wkva = CoderHelpers.fromByteArray(acc, wkvaCoder);
+                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
+                VA va = keyed.addInput(wkva.getValue().getKey(), wkva.getValue().getValue(),
+                    wkvi.getValue().getValue());
+                wkva = WindowedValue.of(KV.of(wkva.getValue().getKey(), va), wkva.getTimestamp(),
+                    wkva.getWindows(), wkva.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            },
+            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[]>() {
+              @Override
+              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc1,
+                  /*KV<K, VA>*/ byte[] acc2) {
+                WindowedValue<KV<K, VA>> wkva1 = CoderHelpers.fromByteArray(acc1, wkvaCoder);
+                WindowedValue<KV<K, VA>> wkva2 = CoderHelpers.fromByteArray(acc2, wkvaCoder);
+                VA va = keyed.mergeAccumulators(wkva1.getValue().getKey(),
+                    // don't use Guava's ImmutableList.of as values may be null
+                    Collections.unmodifiableList(Arrays.asList(wkva1.getValue().getValue(),
+                    wkva2.getValue().getValue())));
+                WindowedValue<KV<K, VA>> wkva = WindowedValue.of(KV.of(wkva1.getValue().getKey(),
+                    va), wkva1.getTimestamp(), wkva1.getWindows(), wkva1.getPane());
+                return CoderHelpers.toByteArray(wkva, wkvaCoder);
+              }
+            });
+
+        JavaPairRDD<WindowedValue<K>, WindowedValue<VO>> extracted = accumulatedBytes
+            .mapToPair(CoderHelpers.fromByteFunction(wkCoder, wkvaCoder))
+            .mapValues(
+                new Function<WindowedValue<KV<K, VA>>, WindowedValue<VO>>() {
+                  @Override
+                  public WindowedValue<VO> call(WindowedValue<KV<K, VA>> acc) {
+                    return WindowedValue.of(keyed.extractOutput(acc.getValue().getKey(),
+                        acc.getValue().getValue()), acc.getTimestamp(),
+                        acc.getWindows(), acc.getPane());
+                  }
+                });
+
+        context.setOutputRDD(transform,
+            fromPair(extracted)
+            .map(new Function<KV<WindowedValue<K>, WindowedValue<VO>>, WindowedValue<KV<K, VO>>>() {
+              @Override
+              public WindowedValue<KV<K, VO>> call(KV<WindowedValue<K>, WindowedValue<VO>> kwvo)
+                  throws Exception {
+                WindowedValue<VO> wvo = kwvo.getValue();
+                KV<K, VO> kvo = KV.of(kwvo.getKey().getValue(), wvo.getValue());
+                return WindowedValue.of(kvo, wvo.getTimestamp(), wvo.getWindows(), wvo.getPane());
+              }
+            }));
+      }
+    };
+  }
+
+  private static final class KVFunction<K, VI, VO>
+      implements Function<WindowedValue<KV<K, Iterable<VI>>>, WindowedValue<KV<K, VO>>> {
+    private final Combine.KeyedCombineFn<K, VI, ?, VO> keyed;
+
+     KVFunction(Combine.KeyedCombineFn<K, VI, ?, VO> keyed) {
+      this.keyed = keyed;
+    }
+
+    @Override
+    public WindowedValue<KV<K, VO>> call(WindowedValue<KV<K, Iterable<VI>>> windowedKv)
+        throws Exception {
+      KV<K, Iterable<VI>> kv = windowedKv.getValue();
+      return WindowedValue.of(KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())),
+          windowedKv.getTimestamp(), windowedKv.getWindows(), windowedKv.getPane());
+    }
+  }
+
+  private static <K, V> JavaPairRDD<K, V> toPair(JavaRDDLike<KV<K, V>, ?> rdd) {
+    return rdd.mapToPair(new PairFunction<KV<K, V>, K, V>() {
+      @Override
+      public Tuple2<K, V> call(KV<K, V> kv) {
+        return new Tuple2<>(kv.getKey(), kv.getValue());
+      }
+    });
+  }
+
+  private static <K, V> JavaRDDLike<KV<K, V>, ?> fromPair(JavaPairRDD<K, V> rdd) {
+    return rdd.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+      @Override
+      public KV<K, V> call(Tuple2<K, V> t2) {
+        return KV.of(t2._1(), t2._2());
+      }
+    });
+  }
+
+  private static <I, O> TransformEvaluator<ParDo.Bound<I, O>> parDo() {
+    return new TransformEvaluator<ParDo.Bound<I, O>>() {
+      @Override
+      public void evaluate(ParDo.Bound<I, O> transform, EvaluationContext context) {
+        DoFnFunction<I, O> dofn =
+            new DoFnFunction<>(transform.getFn(),
+                context.getRuntimeContext(),
+                getSideInputs(transform.getSideInputs(), context));
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+        context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
+      }
+    };
+  }
+
+  private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class);
+
+  private static <I, O> TransformEvaluator<ParDo.BoundMulti<I, O>> multiDo() {
+    return new TransformEvaluator<ParDo.BoundMulti<I, O>>() {
+      @Override
+      public void evaluate(ParDo.BoundMulti<I, O> transform, EvaluationContext context) {
+        TupleTag<O> mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform);
+        MultiDoFnFunction<I, O> multifn = new MultiDoFnFunction<>(
+            transform.getFn(),
+            context.getRuntimeContext(),
+            mainOutputTag,
+            getSideInputs(transform.getSideInputs(), context));
+
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<I>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
+        JavaPairRDD<TupleTag<?>, WindowedValue<?>> all = inRDD
+            .mapPartitionsToPair(multifn)
+            .cache();
+
+        PCollectionTuple pct = context.getOutput(transform);
+        for (Map.Entry<TupleTag<?>, PCollection<?>> e : pct.getAll().entrySet()) {
+          @SuppressWarnings("unchecked")
+          JavaPairRDD<TupleTag<?>, WindowedValue<?>> filtered =
+              all.filter(new TupleTagFilter(e.getKey()));
+          @SuppressWarnings("unchecked")
+          // Object is the best we can do since different outputs can have different tags
+          JavaRDD<WindowedValue<Object>> values =
+              (JavaRDD<WindowedValue<Object>>) (JavaRDD<?>) filtered.values();
+          context.setRDD(e.getValue(), values);
+        }
+      }
+    };
+  }
+
+
+  private static <T> TransformEvaluator<TextIO.Read.Bound<T>> readText() {
+    return new TransformEvaluator<TextIO.Read.Bound<T>>() {
+      @Override
+      public void evaluate(TextIO.Read.Bound<T> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaRDD<WindowedValue<String>> rdd = context.getSparkContext().textFile(pattern)
+                .map(WindowingHelpers.<String>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<TextIO.Write.Bound<T>> writeText() {
+    return new TransformEvaluator<TextIO.Write.Bound<T>>() {
+      @Override
+      public void evaluate(TextIO.Write.Bound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<T, Void> last =
+            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
+            .map(WindowingHelpers.<T>unwindowFunction())
+            .mapToPair(new PairFunction<T, T,
+                    Void>() {
+              @Override
+              public Tuple2<T, Void> call(T t) throws Exception {
+                return new Tuple2<>(t, null);
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+                transform.getShardTemplate(), transform.getFilenamePrefix(),
+                transform.getFilenameSuffix());
+        writeHadoopFile(last, new Configuration(), shardTemplateInfo, Text.class,
+            NullWritable.class, TemplatedTextOutputFormat.class);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<AvroIO.Read.Bound<T>> readAvro() {
+    return new TransformEvaluator<AvroIO.Read.Bound<T>>() {
+      @Override
+      public void evaluate(AvroIO.Read.Bound<T> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaSparkContext jsc = context.getSparkContext();
+        @SuppressWarnings("unchecked")
+        JavaRDD<AvroKey<T>> avroFile = (JavaRDD<AvroKey<T>>) (JavaRDD<?>)
+            jsc.newAPIHadoopFile(pattern,
+                                 AvroKeyInputFormat.class,
+                                 AvroKey.class, NullWritable.class,
+                                 new Configuration()).keys();
+        JavaRDD<WindowedValue<T>> rdd = avroFile.map(
+            new Function<AvroKey<T>, T>() {
+              @Override
+              public T call(AvroKey<T> key) {
+                return key.datum();
+              }
+            }).map(WindowingHelpers.<T>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<AvroIO.Write.Bound<T>> writeAvro() {
+    return new TransformEvaluator<AvroIO.Write.Bound<T>>() {
+      @Override
+      public void evaluate(AvroIO.Write.Bound<T> transform, EvaluationContext context) {
+        Job job;
+        try {
+          job = Job.getInstance();
+        } catch (IOException e) {
+          throw new IllegalStateException(e);
+        }
+        AvroJob.setOutputKeySchema(job, transform.getSchema());
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<AvroKey<T>, NullWritable> last =
+            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
+            .map(WindowingHelpers.<T>unwindowFunction())
+            .mapToPair(new PairFunction<T, AvroKey<T>, NullWritable>() {
+              @Override
+              public Tuple2<AvroKey<T>, NullWritable> call(T t) throws Exception {
+                return new Tuple2<>(new AvroKey<>(t), NullWritable.get());
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+            transform.getShardTemplate(), transform.getFilenamePrefix(),
+            transform.getFilenameSuffix());
+        writeHadoopFile(last, job.getConfiguration(), shardTemplateInfo,
+            AvroKey.class, NullWritable.class, TemplatedAvroKeyOutputFormat.class);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<HadoopIO.Read.Bound<K, V>> readHadoop() {
+    return new TransformEvaluator<HadoopIO.Read.Bound<K, V>>() {
+      @Override
+      public void evaluate(HadoopIO.Read.Bound<K, V> transform, EvaluationContext context) {
+        String pattern = transform.getFilepattern();
+        JavaSparkContext jsc = context.getSparkContext();
+        @SuppressWarnings ("unchecked")
+        JavaPairRDD<K, V> file = jsc.newAPIHadoopFile(pattern,
+            transform.getFormatClass(),
+            transform.getKeyClass(), transform.getValueClass(),
+            new Configuration());
+        JavaRDD<WindowedValue<KV<K, V>>> rdd =
+            file.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+          @Override
+          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
+            return KV.of(t2._1(), t2._2());
+          }
+        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
+        context.setOutputRDD(transform, rdd);
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<HadoopIO.Write.Bound<K, V>> writeHadoop() {
+    return new TransformEvaluator<HadoopIO.Write.Bound<K, V>>() {
+      @Override
+      public void evaluate(HadoopIO.Write.Bound<K, V> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaPairRDD<K, V> last = ((JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context
+            .getInputRDD(transform))
+            .map(WindowingHelpers.<KV<K, V>>unwindowFunction())
+            .mapToPair(new PairFunction<KV<K, V>, K, V>() {
+              @Override
+              public Tuple2<K, V> call(KV<K, V> t) throws Exception {
+                return new Tuple2<>(t.getKey(), t.getValue());
+              }
+            });
+        ShardTemplateInformation shardTemplateInfo =
+            new ShardTemplateInformation(transform.getNumShards(),
+                transform.getShardTemplate(), transform.getFilenamePrefix(),
+                transform.getFilenameSuffix());
+        Configuration conf = new Configuration();
+        for (Map.Entry<String, String> e : transform.getConfigurationProperties().entrySet()) {
+          conf.set(e.getKey(), e.getValue());
+        }
+        writeHadoopFile(last, conf, shardTemplateInfo,
+            transform.getKeyClass(), transform.getValueClass(), transform.getFormatClass());
+      }
+    };
+  }
+
+  private static final class ShardTemplateInformation {
+    private final int numShards;
+    private final String shardTemplate;
+    private final String filenamePrefix;
+    private final String filenameSuffix;
+
+    private ShardTemplateInformation(int numShards, String shardTemplate, String
+        filenamePrefix, String filenameSuffix) {
+      this.numShards = numShards;
+      this.shardTemplate = shardTemplate;
+      this.filenamePrefix = filenamePrefix;
+      this.filenameSuffix = filenameSuffix;
+    }
+
+    int getNumShards() {
+      return numShards;
+    }
+
+    String getShardTemplate() {
+      return shardTemplate;
+    }
+
+    String getFilenamePrefix() {
+      return filenamePrefix;
+    }
+
+    String getFilenameSuffix() {
+      return filenameSuffix;
+    }
+  }
+
+  private static <K, V> void writeHadoopFile(JavaPairRDD<K, V> rdd, Configuration conf,
+      ShardTemplateInformation shardTemplateInfo, Class<?> keyClass, Class<?> valueClass,
+      Class<? extends FileOutputFormat> formatClass) {
+    int numShards = shardTemplateInfo.getNumShards();
+    String shardTemplate = shardTemplateInfo.getShardTemplate();
+    String filenamePrefix = shardTemplateInfo.getFilenamePrefix();
+    String filenameSuffix = shardTemplateInfo.getFilenameSuffix();
+    if (numShards != 0) {
+      // number of shards was set explicitly, so repartition
+      rdd = rdd.repartition(numShards);
+    }
+    int actualNumShards = rdd.partitions().size();
+    String template = replaceShardCount(shardTemplate, actualNumShards);
+    String outputDir = getOutputDirectory(filenamePrefix, template);
+    String filePrefix = getOutputFilePrefix(filenamePrefix, template);
+    String fileTemplate = getOutputFileTemplate(filenamePrefix, template);
+
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_PREFIX, filePrefix);
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_TEMPLATE, fileTemplate);
+    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_SUFFIX, filenameSuffix);
+    rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
+  }
+
+  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
+
+  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
+    return new TransformEvaluator<Window.Bound<T>>() {
+      @Override
+      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaRDDLike<WindowedValue<T>, ?> inRDD =
+            (JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform);
+        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
+        if (windowFn instanceof GlobalWindows) {
+          context.setOutputRDD(transform, inRDD);
+        } else {
+          @SuppressWarnings("unchecked")
+          DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
+          DoFnFunction<T, T> dofn =
+                  new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null);
+          context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
+        }
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Create.Values<T>> create() {
+    return new TransformEvaluator<Create.Values<T>>() {
+      @Override
+      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
+        Iterable<T> elems = transform.getElements();
+        // Use a coder to convert the objects in the PCollection to byte arrays, so they
+        // can be transferred over the network.
+        Coder<T> coder = context.getOutput(transform).getCoder();
+        context.setOutputRDDFromValues(transform, elems, coder);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<View.AsSingleton<T>> viewAsSingleton() {
+    return new TransformEvaluator<View.AsSingleton<T>>() {
+      @Override
+      public void evaluate(View.AsSingleton<T> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+                context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<View.AsIterable<T>> viewAsIter() {
+    return new TransformEvaluator<View.AsIterable<T>>() {
+      @Override
+      public void evaluate(View.AsIterable<T> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+                context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static <R, W> TransformEvaluator<View.CreatePCollectionView<R, W>> createPCollView() {
+    return new TransformEvaluator<View.CreatePCollectionView<R, W>>() {
+      @Override
+      public void evaluate(View.CreatePCollectionView<R, W> transform, EvaluationContext context) {
+        Iterable<? extends WindowedValue<?>> iter =
+            context.getWindowedValues(context.getInput(transform));
+        context.setPView(context.getOutput(transform), iter);
+      }
+    };
+  }
+
+  private static final class TupleTagFilter<V>
+      implements Function<Tuple2<TupleTag<V>, WindowedValue<?>>, Boolean> {
+
+    private final TupleTag<V> tag;
+
+    private TupleTagFilter(TupleTag<V> tag) {
+      this.tag = tag;
+    }
+
+    @Override
+    public Boolean call(Tuple2<TupleTag<V>, WindowedValue<?>> input) {
+      return tag.equals(input._1());
+    }
+  }
+
+  private static Map<TupleTag<?>, BroadcastHelper<?>> getSideInputs(
+      List<PCollectionView<?>> views,
+      EvaluationContext context) {
+    if (views == null) {
+      return ImmutableMap.of();
+    } else {
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs = Maps.newHashMap();
+      for (PCollectionView<?> view : views) {
+        Iterable<? extends WindowedValue<?>> collectionView = context.getPCollectionView(view);
+        Coder<Iterable<WindowedValue<?>>> coderInternal = view.getCoderInternal();
+        @SuppressWarnings("unchecked")
+        BroadcastHelper<?> helper =
+            BroadcastHelper.create((Iterable<WindowedValue<?>>) collectionView, coderInternal);
+        //broadcast side inputs
+        helper.broadcast(context.getSparkContext());
+        sideInputs.put(view.getTagInternal(), helper);
+      }
+      return sideInputs;
+    }
+  }
+
+  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
+      .newHashMap();
+
+  static {
+    EVALUATORS.put(TextIO.Read.Bound.class, readText());
+    EVALUATORS.put(TextIO.Write.Bound.class, writeText());
+    EVALUATORS.put(AvroIO.Read.Bound.class, readAvro());
+    EVALUATORS.put(AvroIO.Write.Bound.class, writeAvro());
+    EVALUATORS.put(HadoopIO.Read.Bound.class, readHadoop());
+    EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop());
+    EVALUATORS.put(ParDo.Bound.class, parDo());
+    EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
+    EVALUATORS.put(GroupByKey.GroupByKeyOnly.class, gbk());
+    EVALUATORS.put(Combine.GroupedValues.class, grouped());
+    EVALUATORS.put(Combine.Globally.class, combineGlobally());
+    EVALUATORS.put(Combine.PerKey.class, combinePerKey());
+    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
+    EVALUATORS.put(Create.Values.class, create());
+    EVALUATORS.put(View.AsSingleton.class, viewAsSingleton());
+    EVALUATORS.put(View.AsIterable.class, viewAsIter());
+    EVALUATORS.put(View.CreatePCollectionView.class, createPCollView());
+    EVALUATORS.put(Window.Bound.class, window());
+  }
+
+  public static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
+  getTransformEvaluator(Class<PT> clazz) {
+    @SuppressWarnings("unchecked")
+    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
+    if (transform == null) {
+      throw new IllegalStateException("No TransformEvaluator registered for " + clazz);
+    }
+    return transform;
+  }
+
+  /**
+   * Translator matches Dataflow transformation with the appropriate evaluator.
+   */
+  public static class Translator implements SparkPipelineTranslator {
+
+    @Override
+    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
+      return EVALUATORS.containsKey(clazz);
+    }
+
+    @Override
+    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
+      return getTransformEvaluator(clazz);
+    }
+  }
+}



[13/23] incubator-beam git commit: [BEAM-11] add Spark runner to included runners

Posted by am...@apache.org.
[BEAM-11] add Spark runner to included runners


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

Branch: refs/heads/master
Commit: b49e3c95bca2de7ce78fbe7ab3cf0148b3a574ab
Parents: eb0341d
Author: Sela <an...@paypal.com>
Authored: Mon Mar 14 23:53:09 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:40:18 2016 +0200

----------------------------------------------------------------------
 README.md | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b49e3c95/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 1a089ff..4e6b995 100644
--- a/README.md
+++ b/README.md
@@ -49,9 +49,7 @@ Beam supports executing programs on multiple distributed processing backends thr
 - The `DirectPipelineRunner` runs the pipeline on your local machine.
 - The `DataflowPipelineRunner` submits the pipeline to the [Google Cloud Dataflow](http://cloud.google.com/dataflow/).
 - The `FlinkPipelineRunner` runs the pipeline on an Apache Flink cluster. The code has been donated from [dataArtisans/flink-dataflow](https://github.com/dataArtisans/flink-dataflow) and is now part of Beam.
-
-After the Beam project's initial bootstrapping completes, it will include:
-- The `SparkPipelineRunner` runs the pipeline on an Apache Spark cluster. See the code that will be donated at [cloudera/spark-dataflow](https://github.com/cloudera/spark-dataflow).
+- The `SparkPipelineRunner` runs the pipeline on an Apache Spark cluster. The code has been donated from [cloudera/spark-dataflow](https://github.com/cloudera/spark-dataflow) and is now part of Beam.
 
 Have ideas for new Runners? See the [Jira](https://issues.apache.org/jira/browse/BEAM/component/12328916/).
 
@@ -64,6 +62,9 @@ _Coming soon!_
 
 See the Flink Runner [README](https://github.com/apache/incubator-beam/tree/master/runners/flink).
 
+### Spark Runner
+
+See the Spark Runner [README](https://github.com/apache/incubator-beam/tree/master/runners/spark).
 
 ## Contact Us
 


[15/23] incubator-beam git commit: [BEAM-11] second iteration of package reorganisation

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
deleted file mode 100644
index 05340d6..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import org.apache.beam.runners.spark.io.KafkaIO;
-import org.apache.beam.runners.spark.EvaluationResult;
-import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
-import org.apache.beam.runners.spark.streaming.utils.EmbeddedKafkaCluster;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.joda.time.Duration;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import kafka.serializer.StringDecoder;
-
-/**
- * Test Kafka as input.
- */
-public class KafkaStreamingTest {
-  private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
-          new EmbeddedKafkaCluster.EmbeddedZookeeper(17001);
-  private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER =
-          new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(),
-                  new Properties(), Collections.singletonList(6667));
-  private static final String TOPIC = "kafka_dataflow_test_topic";
-  private static final Map<String, String> KAFKA_MESSAGES = ImmutableMap.of(
-      "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
-  );
-  private static final Set<String> EXPECTED = ImmutableSet.of(
-      "k1,v1", "k2,v2", "k3,v3", "k4,v4"
-  );
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @BeforeClass
-  public static void init() throws IOException {
-    EMBEDDED_ZOOKEEPER.startup();
-    EMBEDDED_KAFKA_CLUSTER.startup();
-
-    // write to Kafka
-    Properties producerProps = new Properties();
-    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
-    producerProps.put("request.required.acks", 1);
-    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
-    Serializer<String> stringSerializer = new StringSerializer();
-    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
-        new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
-      for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
-        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
-      }
-    }
-  }
-
-  @Test
-  public void testRun() throws Exception {
-    // test read from Kafka
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    Map<String, String> kafkaParams = ImmutableMap.of(
-        "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
-        "auto.offset.reset", "smallest"
-    );
-
-    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
-        StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
-        kafkaParams))
-        .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-    PCollection<KV<String, String>> windowedWords = kafkaInput
-        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
-
-    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn()));
-
-    DataflowAssert.thatIterable(formattedKV.apply(View.<String>asIterable()))
-        .containsInAnyOrder(EXPECTED);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    EMBEDDED_KAFKA_CLUSTER.shutdown();
-    EMBEDDED_ZOOKEEPER.shutdown();
-  }
-
-  private static class FormatKVFn extends DoFn<KV<String, String>, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element().getKey() + "," + c.element().getValue());
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
deleted file mode 100644
index 16b145a..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.beam.runners.spark.io.CreateStream;
-import org.apache.beam.runners.spark.EvaluationResult;
-import org.apache.beam.runners.spark.SimpleWordCountTest;
-import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
-
-import org.joda.time.Duration;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class SimpleStreamingWordCountTest {
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"};
-  private static final List<Iterable<String>> WORDS_QUEUE =
-      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY));
-  private static final Set<String> EXPECTED_COUNT_SET =
-      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @Test
-  public void testRun() throws Exception {
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<String> inputWords =
-        p.apply(CreateStream.fromQueue(WORDS_QUEUE)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedWords = inputWords
-        .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-
-    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
-
-    DataflowAssert.thatIterable(output.apply(View.<String>asIterable()))
-        .containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
deleted file mode 100644
index 367a062..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming.utils;
-
-import org.apache.beam.runners.spark.EvaluationResult;
-
-import org.junit.Assert;
-
-/**
- * Since DataflowAssert doesn't propagate assert exceptions, use Aggregators to assert streaming
- * success/failure counters.
- */
-public final class DataflowAssertStreaming {
-  /**
-   * Copied aggregator names from {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert}
-   */
-  static final String SUCCESS_COUNTER = "DataflowAssertSuccess";
-  static final String FAILURE_COUNTER = "DataflowAssertFailure";
-
-  private DataflowAssertStreaming() {
-  }
-
-  public static void assertNoFailures(EvaluationResult res) {
-    int failures = res.getAggregatorValue(FAILURE_COUNTER, Integer.class);
-    Assert.assertEquals("Found " + failures + " failures, see the log for details", 0, failures);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
deleted file mode 100644
index 8273684..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
+++ /dev/null
@@ -1,317 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming.utils;
-
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * https://gist.github.com/fjavieralba/7930018
- */
-public class EmbeddedKafkaCluster {
-
-  private static final Logger LOG = LoggerFactory.getLogger(EmbeddedKafkaCluster.class);
-
-  private final List<Integer> ports;
-  private final String zkConnection;
-  private final Properties baseProperties;
-
-  private final String brokerList;
-
-  private final List<KafkaServer> brokers;
-  private final List<File> logDirs;
-
-  public EmbeddedKafkaCluster(String zkConnection) {
-    this(zkConnection, new Properties());
-  }
-
-  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties) {
-    this(zkConnection, baseProperties, Collections.singletonList(-1));
-  }
-
-  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties, List<Integer> ports) {
-    this.zkConnection = zkConnection;
-    this.ports = resolvePorts(ports);
-    this.baseProperties = baseProperties;
-
-    this.brokers = new ArrayList<>();
-    this.logDirs = new ArrayList<>();
-
-    this.brokerList = constructBrokerList(this.ports);
-  }
-
-  private static List<Integer> resolvePorts(List<Integer> ports) {
-    List<Integer> resolvedPorts = new ArrayList<>();
-    for (Integer port : ports) {
-      resolvedPorts.add(resolvePort(port));
-    }
-    return resolvedPorts;
-  }
-
-  private static int resolvePort(int port) {
-    if (port == -1) {
-      return TestUtils.getAvailablePort();
-    }
-    return port;
-  }
-
-  private static String constructBrokerList(List<Integer> ports) {
-    StringBuilder sb = new StringBuilder();
-    for (Integer port : ports) {
-      if (sb.length() > 0) {
-        sb.append(",");
-      }
-      sb.append("localhost:").append(port);
-    }
-    return sb.toString();
-  }
-
-  public void startup() {
-    for (int i = 0; i < ports.size(); i++) {
-      Integer port = ports.get(i);
-      File logDir = TestUtils.constructTempDir("kafka-local");
-
-      Properties properties = new Properties();
-      properties.putAll(baseProperties);
-      properties.setProperty("zookeeper.connect", zkConnection);
-      properties.setProperty("broker.id", String.valueOf(i + 1));
-      properties.setProperty("host.name", "localhost");
-      properties.setProperty("port", Integer.toString(port));
-      properties.setProperty("log.dir", logDir.getAbsolutePath());
-      properties.setProperty("log.flush.interval.messages", String.valueOf(1));
-
-      KafkaServer broker = startBroker(properties);
-
-      brokers.add(broker);
-      logDirs.add(logDir);
-    }
-  }
-
-
-  private static KafkaServer startBroker(Properties props) {
-    KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
-    server.startup();
-    return server;
-  }
-
-  public Properties getProps() {
-    Properties props = new Properties();
-    props.putAll(baseProperties);
-    props.put("metadata.broker.list", brokerList);
-    props.put("zookeeper.connect", zkConnection);
-    return props;
-  }
-
-  public String getBrokerList() {
-    return brokerList;
-  }
-
-  public List<Integer> getPorts() {
-    return ports;
-  }
-
-  public String getZkConnection() {
-    return zkConnection;
-  }
-
-  public void shutdown() {
-    for (KafkaServer broker : brokers) {
-      try {
-        broker.shutdown();
-      } catch (Exception e) {
-        LOG.warn("{}", e.getMessage(), e);
-      }
-    }
-    for (File logDir : logDirs) {
-      try {
-        TestUtils.deleteFile(logDir);
-      } catch (FileNotFoundException e) {
-        LOG.warn("{}", e.getMessage(), e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "EmbeddedKafkaCluster{" + "brokerList='" + brokerList + "'}";
-  }
-
-  public static class EmbeddedZookeeper {
-    private int port = -1;
-    private int tickTime = 500;
-
-    private ServerCnxnFactory factory;
-    private File snapshotDir;
-    private File logDir;
-
-    public EmbeddedZookeeper() {
-      this(-1);
-    }
-
-    public EmbeddedZookeeper(int port) {
-      this(port, 500);
-    }
-
-    public EmbeddedZookeeper(int port, int tickTime) {
-      this.port = resolvePort(port);
-      this.tickTime = tickTime;
-    }
-
-    private static int resolvePort(int port) {
-      if (port == -1) {
-        return TestUtils.getAvailablePort();
-      }
-      return port;
-    }
-
-    public void startup() throws IOException {
-      if (this.port == -1) {
-        this.port = TestUtils.getAvailablePort();
-      }
-      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
-              1024);
-      this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
-      this.logDir = TestUtils.constructTempDir("embedded-zk/log");
-
-      try {
-        factory.startup(new ZooKeeperServer(snapshotDir, logDir, tickTime));
-      } catch (InterruptedException e) {
-        throw new IOException(e);
-      }
-    }
-
-
-    public void shutdown() {
-      factory.shutdown();
-      try {
-        TestUtils.deleteFile(snapshotDir);
-      } catch (FileNotFoundException e) {
-        // ignore
-      }
-      try {
-        TestUtils.deleteFile(logDir);
-      } catch (FileNotFoundException e) {
-        // ignore
-      }
-    }
-
-    public String getConnection() {
-      return "localhost:" + port;
-    }
-
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    public void setTickTime(int tickTime) {
-      this.tickTime = tickTime;
-    }
-
-    public int getPort() {
-      return port;
-    }
-
-    public int getTickTime() {
-      return tickTime;
-    }
-
-    @Override
-    public String toString() {
-      return "EmbeddedZookeeper{" + "connection=" + getConnection() + "}";
-    }
-  }
-
-  static class SystemTime implements Time {
-    @Override
-    public long milliseconds() {
-      return System.currentTimeMillis();
-    }
-
-    @Override
-    public long nanoseconds() {
-      return System.nanoTime();
-    }
-
-    @Override
-    public void sleep(long ms) {
-      try {
-        Thread.sleep(ms);
-      } catch (InterruptedException e) {
-        // Ignore
-      }
-    }
-  }
-
-  static final class TestUtils {
-    private static final Random RANDOM = new Random();
-
-    private TestUtils() {
-    }
-
-    static File constructTempDir(String dirPrefix) {
-      File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt
-              (10000000));
-      if (!file.mkdirs()) {
-        throw new RuntimeException("could not create temp directory: " + file.getAbsolutePath());
-      }
-      file.deleteOnExit();
-      return file;
-    }
-
-    static int getAvailablePort() {
-      try {
-        try (ServerSocket socket = new ServerSocket(0)) {
-          return socket.getLocalPort();
-        }
-      } catch (IOException e) {
-        throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e);
-      }
-    }
-
-    static boolean deleteFile(File path) throws FileNotFoundException {
-      if (!path.exists()) {
-        throw new FileNotFoundException(path.getAbsolutePath());
-      }
-      boolean ret = true;
-      if (path.isDirectory()) {
-        for (File f : path.listFiles()) {
-          ret = ret && deleteFile(f);
-        }
-      }
-      return ret && path.delete();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
new file mode 100644
index 0000000..6945d68
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class CombineGloballyTest {
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+    PCollection<String> output = inputWords.apply(Combine.globally(new WordMerger()));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    assertEquals("hi there,hi,hi sue bob,hi sue,,bob hi", Iterables.getOnlyElement(res.get(output)));
+    res.close();
+  }
+
+  public static class WordMerger extends Combine.CombineFn<String, StringBuilder, String> {
+
+    @Override
+    public StringBuilder createAccumulator() {
+      // return null to differentiate from an empty string
+      return null;
+    }
+
+    @Override
+    public StringBuilder addInput(StringBuilder accumulator, String input) {
+      return combine(accumulator, input);
+    }
+
+    @Override
+    public StringBuilder mergeAccumulators(Iterable<StringBuilder> accumulators) {
+      StringBuilder sb = new StringBuilder();
+      for (StringBuilder accum : accumulators) {
+        if (accum != null) {
+          sb.append(accum);
+        }
+      }
+      return sb;
+    }
+
+    @Override
+    public String extractOutput(StringBuilder accumulator) {
+      return accumulator != null ? accumulator.toString(): "";
+    }
+
+    private static StringBuilder combine(StringBuilder accum, String datum) {
+      if (accum == null) {
+        return new StringBuilder(datum);
+      } else {
+        accum.append(",").append(datum);
+        return accum;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
new file mode 100644
index 0000000..0373968
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CombinePerKeyTest {
+
+    private static final List<String> WORDS =
+        ImmutableList.of("the", "quick", "brown", "fox", "jumped", "over", "the", "lazy", "dog");
+    @Test
+    public void testRun() {
+        Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+        PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+        PCollection<KV<String, Long>> cnts = inputWords.apply(new SumPerKey<String>());
+        EvaluationResult res = SparkPipelineRunner.create().run(p);
+        Map<String, Long> actualCnts = new HashMap<>();
+        for (KV<String, Long> kv : res.get(cnts)) {
+            actualCnts.put(kv.getKey(), kv.getValue());
+        }
+        res.close();
+        Assert.assertEquals(8, actualCnts.size());
+        Assert.assertEquals(Long.valueOf(2L), actualCnts.get("the"));
+    }
+
+    private static class SumPerKey<T> extends PTransform<PCollection<T>, PCollection<KV<T, Long>>> {
+      @Override
+      public PCollection<KV<T, Long>> apply(PCollection<T> pcol) {
+          PCollection<KV<T, Long>> withLongs = pcol.apply(ParDo.of(new DoFn<T, KV<T, Long>>() {
+              @Override
+              public void processElement(ProcessContext processContext) throws Exception {
+                  processContext.output(KV.of(processContext.element(), 1L));
+              }
+          })).setCoder(KvCoder.of(pcol.getCoder(), VarLongCoder.of()));
+          return withLongs.apply(Sum.<T>longsPerKey());
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
new file mode 100644
index 0000000..a9779e6
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Test;
+
+import java.io.Serializable;
+
+public class DoFnOutputTest implements Serializable {
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    PCollection<String> strings = pipeline.apply(Create.of("a"));
+    // Test that values written from startBundle() and finishBundle() are written to
+    // the output
+    PCollection<String> output = strings.apply(ParDo.of(new DoFn<String, String>() {
+      @Override
+      public void startBundle(Context c) throws Exception {
+        c.output("start");
+      }
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(c.element());
+      }
+      @Override
+      public void finishBundle(Context c) throws Exception {
+        c.output("finish");
+      }
+    }));
+
+    DataflowAssert.that(output).containsInAnyOrder("start", "a", "finish");
+
+    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
+    res.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
new file mode 100644
index 0000000..8ab3798
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.*;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MultiOutputWordCountTest {
+
+  private static final TupleTag<String> upper = new TupleTag<>();
+  private static final TupleTag<String> lower = new TupleTag<>();
+  private static final TupleTag<KV<String, Long>> lowerCnts = new TupleTag<>();
+  private static final TupleTag<KV<String, Long>> upperCnts = new TupleTag<>();
+
+  @Test
+  public void testRun() throws Exception {
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    PCollection<String> regex = p.apply(Create.of("[^a-zA-Z']+"));
+    PCollection<String> w1 = p.apply(Create.of("Here are some words to count", "and some others"));
+    PCollection<String> w2 = p.apply(Create.of("Here are some more words", "and even more words"));
+    PCollectionList<String> list = PCollectionList.of(w1).and(w2);
+
+    PCollection<String> union = list.apply(Flatten.<String>pCollections());
+    PCollectionView<String> regexView = regex.apply(View.<String>asSingleton());
+    CountWords countWords = new CountWords(regexView);
+    PCollectionTuple luc = union.apply(countWords);
+    PCollection<Long> unique = luc.get(lowerCnts).apply(
+        ApproximateUnique.<KV<String, Long>>globally(16));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    Iterable<KV<String, Long>> actualLower = res.get(luc.get(lowerCnts));
+    Assert.assertEquals("are", actualLower.iterator().next().getKey());
+    Iterable<KV<String, Long>> actualUpper = res.get(luc.get(upperCnts));
+    Assert.assertEquals("Here", actualUpper.iterator().next().getKey());
+    Iterable<Long> actualUniqCount = res.get(unique);
+    Assert.assertEquals(9, (long) actualUniqCount.iterator().next());
+    int actualTotalWords = res.getAggregatorValue("totalWords", Integer.class);
+    Assert.assertEquals(18, actualTotalWords);
+    int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class);
+    Assert.assertEquals(6, actualMaxWordLength);
+    AggregatorValues<Integer> aggregatorValues = res.getAggregatorValues(countWords
+        .getTotalWordsAggregator());
+    Assert.assertEquals(18, Iterables.getOnlyElement(aggregatorValues.getValues()).intValue());
+
+    res.close();
+  }
+
+  /**
+   * A DoFn that tokenizes lines of text into individual words.
+   */
+  static class ExtractWordsFn extends DoFn<String, String> {
+
+    private final Aggregator<Integer, Integer> totalWords = createAggregator("totalWords",
+        new Sum.SumIntegerFn());
+    private final Aggregator<Integer, Integer> maxWordLength = createAggregator("maxWordLength",
+        new Max.MaxIntegerFn());
+    private final PCollectionView<String> regex;
+
+    ExtractWordsFn(PCollectionView<String> regex) {
+      this.regex = regex;
+    }
+
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] words = c.element().split(c.sideInput(regex));
+      for (String word : words) {
+        totalWords.addValue(1);
+        if (!word.isEmpty()) {
+          maxWordLength.addValue(word.length());
+          if (Character.isLowerCase(word.charAt(0))) {
+            c.output(word);
+          } else {
+            c.sideOutput(upper, word);
+          }
+        }
+      }
+    }
+  }
+
+  public static class CountWords extends PTransform<PCollection<String>, PCollectionTuple> {
+
+    private final PCollectionView<String> regex;
+    private final ExtractWordsFn extractWordsFn;
+
+    public CountWords(PCollectionView<String> regex) {
+      this.regex = regex;
+      this.extractWordsFn = new ExtractWordsFn(regex);
+    }
+
+    @Override
+    public PCollectionTuple apply(PCollection<String> lines) {
+      // Convert lines of text into individual words.
+      PCollectionTuple lowerUpper = lines
+          .apply(ParDo.of(extractWordsFn)
+              .withSideInputs(regex)
+              .withOutputTags(lower, TupleTagList.of(upper)));
+      lowerUpper.get(lower).setCoder(StringUtf8Coder.of());
+      lowerUpper.get(upper).setCoder(StringUtf8Coder.of());
+      PCollection<KV<String, Long>> lowerCounts = lowerUpper.get(lower).apply(Count
+          .<String>perElement());
+      PCollection<KV<String, Long>> upperCounts = lowerUpper.get(upper).apply(Count
+          .<String>perElement());
+      return PCollectionTuple
+          .of(lowerCnts, lowerCounts)
+          .and(upperCnts, upperCounts);
+    }
+
+    Aggregator<Integer, Integer> getTotalWordsAggregator() {
+      return extractWordsFn.totalWords;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
new file mode 100644
index 0000000..b378795
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class SerializationTest {
+
+  public static class StringHolder { // not serializable
+    private final String string;
+
+    public StringHolder(String string) {
+      this.string = string;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      StringHolder that = (StringHolder) o;
+      return string.equals(that.string);
+    }
+
+    @Override
+    public int hashCode() {
+      return string.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return string;
+    }
+  }
+
+  public static class StringHolderUtf8Coder extends AtomicCoder<StringHolder> {
+
+    private final StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of();
+
+    @Override
+    public void encode(StringHolder value, OutputStream outStream, Context context) throws IOException {
+      stringUtf8Coder.encode(value.toString(), outStream, context);
+    }
+
+    @Override
+    public StringHolder decode(InputStream inStream, Context context) throws IOException {
+      return new StringHolder(stringUtf8Coder.decode(inStream, context));
+    }
+
+    public static Coder<StringHolder> of() {
+      return new StringHolderUtf8Coder();
+    }
+  }
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<StringHolder> WORDS = Lists.transform(
+      Arrays.asList(WORDS_ARRAY), new Function<String, StringHolder>() {
+        @Override public StringHolder apply(String s) {
+          return new StringHolder(s);
+        }
+      });
+  private static final Set<StringHolder> EXPECTED_COUNT_SET =
+      ImmutableSet.copyOf(Lists.transform(
+          Arrays.asList("hi: 5", "there: 1", "sue: 2", "bob: 2"),
+          new Function<String, StringHolder>() {
+            @Override
+            public StringHolder apply(String s) {
+              return new StringHolder(s);
+            }
+          }));
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(options);
+    PCollection<StringHolder> inputWords =
+        p.apply(Create.of(WORDS).withCoder(StringHolderUtf8Coder.of()));
+    PCollection<StringHolder> output = inputWords.apply(new CountWords());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+
+  /**
+   * A DoFn that tokenizes lines of text into individual words.
+   */
+  static class ExtractWordsFn extends DoFn<StringHolder, StringHolder> {
+    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
+    private final Aggregator<Long, Long> emptyLines =
+        createAggregator("emptyLines", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      // Split the line into words.
+      String[] words = WORD_BOUNDARY.split(c.element().toString());
+
+      // Keep track of the number of lines without any words encountered while tokenizing.
+      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
+      if (words.length == 0) {
+        emptyLines.addValue(1L);
+      }
+
+      // Output each word encountered into the output PCollection.
+      for (String word : words) {
+        if (!word.isEmpty()) {
+          c.output(new StringHolder(word));
+        }
+      }
+    }
+  }
+
+  /**
+   * A DoFn that converts a Word and Count into a printable string.
+   */
+  private static class FormatCountsFn extends DoFn<KV<StringHolder, Long>, StringHolder> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(new StringHolder(c.element().getKey() + ": " + c.element().getValue()));
+    }
+  }
+
+  private static class CountWords extends PTransform<PCollection<StringHolder>, PCollection<StringHolder>> {
+    @Override
+    public PCollection<StringHolder> apply(PCollection<StringHolder> lines) {
+
+      // Convert lines of text into individual words.
+      PCollection<StringHolder> words = lines.apply(
+          ParDo.of(new ExtractWordsFn()));
+
+      // Count the number of times each word occurs.
+      PCollection<KV<StringHolder, Long>> wordCounts =
+          words.apply(Count.<StringHolder>perElement());
+
+      // Format each word and count into a printable string.
+
+      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
new file mode 100644
index 0000000..fc14fc7
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.net.URI;
+
+import static org.junit.Assert.*;
+
+public class SideEffectsTest implements Serializable {
+
+  static class UserException extends RuntimeException {
+  }
+
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+    pipeline.apply(Create.of("a")).apply(ParDo.of(new DoFn<String, String>() {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        throw new UserException();
+      }
+    }));
+
+    try {
+      pipeline.run();
+      fail("Run should thrown an exception");
+    } catch (RuntimeException e) {
+      assertNotNull(e.getCause());
+
+      // TODO: remove the version check (and the setup and teardown methods) when we no
+      // longer support Spark 1.3 or 1.4
+      String version = SparkContextFactory.getSparkContext(options.getSparkMaster(), options.getAppName()).version();
+      if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) {
+        assertTrue(e.getCause() instanceof UserException);
+      }
+    }
+  }
+
+  @Before
+  public void setup() {
+    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "true");
+  }
+
+  @After
+  public void teardown() {
+    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "false");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TestSparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TestSparkPipelineOptionsFactory.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TestSparkPipelineOptionsFactory.java
new file mode 100644
index 0000000..9cace83
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TestSparkPipelineOptionsFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkPipelineOptionsFactory {
+  @Test
+  public void testDefaultCreateMethod() {
+    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
+    Assert.assertEquals("local[1]", actualOptions.getSparkMaster());
+  }
+
+  @Test
+  public void testSettingCustomOptions() {
+    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
+    actualOptions.setSparkMaster("spark://207.184.161.138:7077");
+    Assert.assertEquals("spark://207.184.161.138:7077", actualOptions.getSparkMaster());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
new file mode 100644
index 0000000..da30321
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.api.client.repackaged.com.google.common.base.Joiner;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Charsets;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A test for the transforms registered in TransformTranslator.
+ * Builds a regular Dataflow pipeline with each of the mapped
+ * transforms, and makes sure that they work when the pipeline is
+ * executed in Spark.
+ */
+public class TransformTranslatorTest {
+
+  @Rule
+  public TestName name = new TestName();
+
+  private DirectPipelineRunner directRunner;
+  private SparkPipelineRunner sparkRunner;
+  private String testDataDirName;
+
+  @Before public void init() throws IOException {
+    sparkRunner = SparkPipelineRunner.create();
+    directRunner = DirectPipelineRunner.createForTest();
+    testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName())
+        + File.separator;
+    FileUtils.deleteDirectory(new File(testDataDirName));
+    new File(testDataDirName).mkdirs();
+  }
+
+  /**
+   * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline
+   * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
+   * transforms. Finally it makes sure that the results are the same for both runs.
+   */
+  @Test
+  public void testTextIOReadAndWriteTransforms() throws IOException {
+    String directOut = runPipeline("direct", directRunner);
+    String sparkOut = runPipeline("spark", sparkRunner);
+
+    List<String> directOutput =
+        Files.readAllLines(Paths.get(directOut + "-00000-of-00001"), Charsets.UTF_8);
+
+    List<String> sparkOutput =
+        Files.readAllLines(Paths.get(sparkOut + "-00000-of-00001"), Charsets.UTF_8);
+
+    // sort output to get a stable result (PCollections are not ordered)
+    Collections.sort(directOutput);
+    Collections.sort(sparkOutput);
+
+    Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray());
+  }
+
+  private String runPipeline(String name, PipelineRunner<?> runner) {
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out_" + name);
+    PCollection<String> lines =  p.apply(TextIO.Read.from("src/test/resources/test_text.txt"));
+    lines.apply(TextIO.Write.to(outFile));
+    runner.run(p);
+    return outFile;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
new file mode 100644
index 0000000..9f29a37
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SimpleWordCountTest;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+public class WindowedWordCountTest {
+  private static final String[] WORDS_ARRAY = {
+          "hi there", "hi", "hi sue bob",
+          "hi sue", "", "bob hi"};
+  private static final Long[] TIMESTAMPS_ARRAY = {
+          60000L, 60000L, 60000L,
+          120000L, 120000L, 120000L};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+  private static final List<Long> TIMESTAMPS = Arrays.asList(TIMESTAMPS_ARRAY);
+  private static final List<String> EXPECTED_COUNT_SET =
+          ImmutableList.of("hi: 3", "there: 1", "sue: 1", "bob: 1",
+                  "hi: 2", "sue: 1", "bob: 1");
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    PCollection<String> inputWords = p.apply(Create.timestamped(WORDS, TIMESTAMPS))
+            .setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedWords = inputWords
+            .apply(Window.<String>into(FixedWindows.of(Duration.standardMinutes(1))));
+
+    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
new file mode 100644
index 0000000..a3eb301
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.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.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.translation.streaming.utils.DataflowAssertStreaming;
+
+import org.joda.time.Duration;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Test Flatten (union) implementation for streaming.
+ */
+public class FlattenStreamingTest {
+
+  private static final String[] WORDS_ARRAY_1 = {
+      "one", "two", "three", "four"};
+  private static final List<Iterable<String>> WORDS_QUEUE_1 =
+      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_1));
+  private static final String[] WORDS_ARRAY_2 = {
+          "five", "six", "seven", "eight"};
+  private static final List<Iterable<String>> WORDS_QUEUE_2 =
+          Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_2));
+  private static final String[] EXPECTED_UNION = {
+          "one", "two", "three", "four", "five", "six", "seven", "eight"};
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @Test
+  public void testRun() throws Exception {
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    PCollection<String> w1 =
+            p.apply(CreateStream.fromQueue(WORDS_QUEUE_1)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedW1 =
+            w1.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+    PCollection<String> w2 =
+            p.apply(CreateStream.fromQueue(WORDS_QUEUE_2)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedW2 =
+            w2.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+    PCollectionList<String> list = PCollectionList.of(windowedW1).and(windowedW2);
+    PCollection<String> union = list.apply(Flatten.<String>pCollections());
+
+    DataflowAssert.thatIterable(union.apply(View.<String>asIterable()))
+            .containsInAnyOrder(EXPECTED_UNION);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
new file mode 100644
index 0000000..628fe86
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.KafkaIO;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.translation.streaming.utils.DataflowAssertStreaming;
+import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.joda.time.Duration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import kafka.serializer.StringDecoder;
+
+/**
+ * Test Kafka as input.
+ */
+public class KafkaStreamingTest {
+  private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
+          new EmbeddedKafkaCluster.EmbeddedZookeeper(17001);
+  private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER =
+          new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(),
+                  new Properties(), Collections.singletonList(6667));
+  private static final String TOPIC = "kafka_dataflow_test_topic";
+  private static final Map<String, String> KAFKA_MESSAGES = ImmutableMap.of(
+      "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
+  );
+  private static final Set<String> EXPECTED = ImmutableSet.of(
+      "k1,v1", "k2,v2", "k3,v3", "k4,v4"
+  );
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    EMBEDDED_ZOOKEEPER.startup();
+    EMBEDDED_KAFKA_CLUSTER.startup();
+
+    // write to Kafka
+    Properties producerProps = new Properties();
+    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
+    producerProps.put("request.required.acks", 1);
+    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
+    Serializer<String> stringSerializer = new StringSerializer();
+    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
+        new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
+      for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
+        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
+      }
+    }
+  }
+
+  @Test
+  public void testRun() throws Exception {
+    // test read from Kafka
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    Map<String, String> kafkaParams = ImmutableMap.of(
+        "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
+        "auto.offset.reset", "smallest"
+    );
+
+    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
+        StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
+        kafkaParams))
+        .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+    PCollection<KV<String, String>> windowedWords = kafkaInput
+        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
+
+    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn()));
+
+    DataflowAssert.thatIterable(formattedKV.apply(View.<String>asIterable()))
+        .containsInAnyOrder(EXPECTED);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    EMBEDDED_KAFKA_CLUSTER.shutdown();
+    EMBEDDED_ZOOKEEPER.shutdown();
+  }
+
+  private static class FormatKVFn extends DoFn<KV<String, String>, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().getKey() + "," + c.element().getValue());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/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
new file mode 100644
index 0000000..b591510
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SimpleWordCountTest;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.translation.streaming.utils.DataflowAssertStreaming;
+
+import org.joda.time.Duration;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class SimpleStreamingWordCountTest {
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"};
+  private static final List<Iterable<String>> WORDS_QUEUE =
+      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY));
+  private static final Set<String> EXPECTED_COUNT_SET =
+      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @Test
+  public void testRun() throws Exception {
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    PCollection<String> inputWords =
+        p.apply(CreateStream.fromQueue(WORDS_QUEUE)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedWords = inputWords
+        .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+
+    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
+
+    DataflowAssert.thatIterable(output.apply(View.<String>asIterable()))
+        .containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/DataflowAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/DataflowAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/DataflowAssertStreaming.java
new file mode 100644
index 0000000..30673dd
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/DataflowAssertStreaming.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming.utils;
+
+import org.apache.beam.runners.spark.EvaluationResult;
+
+import org.junit.Assert;
+
+/**
+ * Since DataflowAssert doesn't propagate assert exceptions, use Aggregators to assert streaming
+ * success/failure counters.
+ */
+public final class DataflowAssertStreaming {
+  /**
+   * Copied aggregator names from {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert}
+   */
+  static final String SUCCESS_COUNTER = "DataflowAssertSuccess";
+  static final String FAILURE_COUNTER = "DataflowAssertFailure";
+
+  private DataflowAssertStreaming() {
+  }
+
+  public static void assertNoFailures(EvaluationResult res) {
+    int failures = res.getAggregatorValue(FAILURE_COUNTER, Integer.class);
+    Assert.assertEquals("Found " + failures + " failures, see the log for details", 0, failures);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
new file mode 100644
index 0000000..e967cdb
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming.utils;
+
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * https://gist.github.com/fjavieralba/7930018
+ */
+public class EmbeddedKafkaCluster {
+
+  private static final Logger LOG = LoggerFactory.getLogger(EmbeddedKafkaCluster.class);
+
+  private final List<Integer> ports;
+  private final String zkConnection;
+  private final Properties baseProperties;
+
+  private final String brokerList;
+
+  private final List<KafkaServer> brokers;
+  private final List<File> logDirs;
+
+  public EmbeddedKafkaCluster(String zkConnection) {
+    this(zkConnection, new Properties());
+  }
+
+  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties) {
+    this(zkConnection, baseProperties, Collections.singletonList(-1));
+  }
+
+  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties, List<Integer> ports) {
+    this.zkConnection = zkConnection;
+    this.ports = resolvePorts(ports);
+    this.baseProperties = baseProperties;
+
+    this.brokers = new ArrayList<>();
+    this.logDirs = new ArrayList<>();
+
+    this.brokerList = constructBrokerList(this.ports);
+  }
+
+  private static List<Integer> resolvePorts(List<Integer> ports) {
+    List<Integer> resolvedPorts = new ArrayList<>();
+    for (Integer port : ports) {
+      resolvedPorts.add(resolvePort(port));
+    }
+    return resolvedPorts;
+  }
+
+  private static int resolvePort(int port) {
+    if (port == -1) {
+      return TestUtils.getAvailablePort();
+    }
+    return port;
+  }
+
+  private static String constructBrokerList(List<Integer> ports) {
+    StringBuilder sb = new StringBuilder();
+    for (Integer port : ports) {
+      if (sb.length() > 0) {
+        sb.append(",");
+      }
+      sb.append("localhost:").append(port);
+    }
+    return sb.toString();
+  }
+
+  public void startup() {
+    for (int i = 0; i < ports.size(); i++) {
+      Integer port = ports.get(i);
+      File logDir = TestUtils.constructTempDir("kafka-local");
+
+      Properties properties = new Properties();
+      properties.putAll(baseProperties);
+      properties.setProperty("zookeeper.connect", zkConnection);
+      properties.setProperty("broker.id", String.valueOf(i + 1));
+      properties.setProperty("host.name", "localhost");
+      properties.setProperty("port", Integer.toString(port));
+      properties.setProperty("log.dir", logDir.getAbsolutePath());
+      properties.setProperty("log.flush.interval.messages", String.valueOf(1));
+
+      KafkaServer broker = startBroker(properties);
+
+      brokers.add(broker);
+      logDirs.add(logDir);
+    }
+  }
+
+
+  private static KafkaServer startBroker(Properties props) {
+    KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
+    server.startup();
+    return server;
+  }
+
+  public Properties getProps() {
+    Properties props = new Properties();
+    props.putAll(baseProperties);
+    props.put("metadata.broker.list", brokerList);
+    props.put("zookeeper.connect", zkConnection);
+    return props;
+  }
+
+  public String getBrokerList() {
+    return brokerList;
+  }
+
+  public List<Integer> getPorts() {
+    return ports;
+  }
+
+  public String getZkConnection() {
+    return zkConnection;
+  }
+
+  public void shutdown() {
+    for (KafkaServer broker : brokers) {
+      try {
+        broker.shutdown();
+      } catch (Exception e) {
+        LOG.warn("{}", e.getMessage(), e);
+      }
+    }
+    for (File logDir : logDirs) {
+      try {
+        TestUtils.deleteFile(logDir);
+      } catch (FileNotFoundException e) {
+        LOG.warn("{}", e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "EmbeddedKafkaCluster{" + "brokerList='" + brokerList + "'}";
+  }
+
+  public static class EmbeddedZookeeper {
+    private int port = -1;
+    private int tickTime = 500;
+
+    private ServerCnxnFactory factory;
+    private File snapshotDir;
+    private File logDir;
+
+    public EmbeddedZookeeper() {
+      this(-1);
+    }
+
+    public EmbeddedZookeeper(int port) {
+      this(port, 500);
+    }
+
+    public EmbeddedZookeeper(int port, int tickTime) {
+      this.port = resolvePort(port);
+      this.tickTime = tickTime;
+    }
+
+    private static int resolvePort(int port) {
+      if (port == -1) {
+        return TestUtils.getAvailablePort();
+      }
+      return port;
+    }
+
+    public void startup() throws IOException {
+      if (this.port == -1) {
+        this.port = TestUtils.getAvailablePort();
+      }
+      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
+              1024);
+      this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
+      this.logDir = TestUtils.constructTempDir("embedded-zk/log");
+
+      try {
+        factory.startup(new ZooKeeperServer(snapshotDir, logDir, tickTime));
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+    }
+
+
+    public void shutdown() {
+      factory.shutdown();
+      try {
+        TestUtils.deleteFile(snapshotDir);
+      } catch (FileNotFoundException e) {
+        // ignore
+      }
+      try {
+        TestUtils.deleteFile(logDir);
+      } catch (FileNotFoundException e) {
+        // ignore
+      }
+    }
+
+    public String getConnection() {
+      return "localhost:" + port;
+    }
+
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    public void setTickTime(int tickTime) {
+      this.tickTime = tickTime;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    public int getTickTime() {
+      return tickTime;
+    }
+
+    @Override
+    public String toString() {
+      return "EmbeddedZookeeper{" + "connection=" + getConnection() + "}";
+    }
+  }
+
+  static class SystemTime implements Time {
+    @Override
+    public long milliseconds() {
+      return System.currentTimeMillis();
+    }
+
+    @Override
+    public long nanoseconds() {
+      return System.nanoTime();
+    }
+
+    @Override
+    public void sleep(long ms) {
+      try {
+        Thread.sleep(ms);
+      } catch (InterruptedException e) {
+        // Ignore
+      }
+    }
+  }
+
+  static final class TestUtils {
+    private static final Random RANDOM = new Random();
+
+    private TestUtils() {
+    }
+
+    static File constructTempDir(String dirPrefix) {
+      File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt
+              (10000000));
+      if (!file.mkdirs()) {
+        throw new RuntimeException("could not create temp directory: " + file.getAbsolutePath());
+      }
+      file.deleteOnExit();
+      return file;
+    }
+
+    static int getAvailablePort() {
+      try {
+        try (ServerSocket socket = new ServerSocket(0)) {
+          return socket.getLocalPort();
+        }
+      } catch (IOException e) {
+        throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e);
+      }
+    }
+
+    static boolean deleteFile(File path) throws FileNotFoundException {
+      if (!path.exists()) {
+        throw new FileNotFoundException(path.getAbsolutePath());
+      }
+      boolean ret = true;
+      if (path.isDirectory()) {
+        for (File f : path.listFiles()) {
+          ret = ret && deleteFile(f);
+        }
+      }
+      return ret && path.delete();
+    }
+  }
+}



[04/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
new file mode 100644
index 0000000..44d8e0f
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.net.URI;
+
+import static org.junit.Assert.*;
+
+public class SideEffectsTest implements Serializable {
+
+  static class UserException extends RuntimeException {
+  }
+
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+    pipeline.apply(Create.of("a")).apply(ParDo.of(new DoFn<String, String>() {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        throw new UserException();
+      }
+    }));
+
+    try {
+      pipeline.run();
+      fail("Run should thrown an exception");
+    } catch (RuntimeException e) {
+      assertNotNull(e.getCause());
+
+      // TODO: remove the version check (and the setup and teardown methods) when we no
+      // longer support Spark 1.3 or 1.4
+      String version = SparkContextFactory.getSparkContext(options.getSparkMaster(), options.getAppName()).version();
+      if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) {
+        assertTrue(e.getCause() instanceof UserException);
+      }
+    }
+  }
+
+  @Before
+  public void setup() {
+    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "true");
+  }
+
+  @After
+  public void teardown() {
+    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "false");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
new file mode 100644
index 0000000..f930855
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class SimpleWordCountTest {
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+  private static final Set<String> EXPECTED_COUNT_SET =
+      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(options);
+    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder
+        .of());
+    PCollection<String> output = inputWords.apply(new CountWords());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+
+  /**
+   * A DoFn that tokenizes lines of text into individual words.
+   */
+  static class ExtractWordsFn extends DoFn<String, String> {
+    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
+    private final Aggregator<Long, Long> emptyLines =
+        createAggregator("emptyLines", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      // Split the line into words.
+      String[] words = WORD_BOUNDARY.split(c.element());
+
+      // Keep track of the number of lines without any words encountered while tokenizing.
+      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
+      if (words.length == 0) {
+        emptyLines.addValue(1L);
+      }
+
+      // Output each word encountered into the output PCollection.
+      for (String word : words) {
+        if (!word.isEmpty()) {
+          c.output(word);
+        }
+      }
+    }
+  }
+
+  /**
+   * A DoFn that converts a Word and Count into a printable string.
+   */
+  private static class FormatCountsFn extends DoFn<KV<String, Long>, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().getKey() + ": " + c.element().getValue());
+    }
+  }
+
+  public static class CountWords extends PTransform<PCollection<String>, PCollection<String>> {
+    @Override
+    public PCollection<String> apply(PCollection<String> lines) {
+
+      // Convert lines of text into individual words.
+      PCollection<String> words = lines.apply(
+          ParDo.of(new ExtractWordsFn()));
+
+      // Count the number of times each word occurs.
+      PCollection<KV<String, Long>> wordCounts =
+          words.apply(Count.<String>perElement());
+
+      // Format each word and count into a printable string.
+
+      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
new file mode 100644
index 0000000..3fc3ecc
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkPipelineOptionsFactory {
+  @Test
+  public void testDefaultCreateMethod() {
+    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
+    Assert.assertEquals("local[1]", actualOptions.getSparkMaster());
+  }
+
+  @Test
+  public void testSettingCustomOptions() {
+    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
+    actualOptions.setSparkMaster("spark://207.184.161.138:7077");
+    Assert.assertEquals("spark://207.184.161.138:7077", actualOptions.getSparkMaster());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
new file mode 100644
index 0000000..0a36c9e
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.examples.complete.TfIdf;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Keys;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.junit.Test;
+
+import java.net.URI;
+import java.util.Arrays;
+
+/**
+ * A test based on {@code TfIdf} from the SDK.
+ */
+public class TfIdfTest {
+
+  @Test
+  public void testTfIdf() throws Exception {
+    Pipeline pipeline = Pipeline.create(PipelineOptionsFactory.create());
+
+    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+    PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
+        .apply(Create.of(
+            KV.of(new URI("x"), "a b c d"),
+            KV.of(new URI("y"), "a b c"),
+            KV.of(new URI("z"), "a m n")))
+        .apply(new TfIdf.ComputeTfIdf());
+
+    PCollection<String> words = wordToUriAndTfIdf
+        .apply(Keys.<String>create())
+        .apply(RemoveDuplicates.<String>create());
+
+    DataflowAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d"));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
+    res.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
new file mode 100644
index 0000000..f759fe9
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.api.client.repackaged.com.google.common.base.Joiner;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Charsets;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A test for the transforms registered in TransformTranslator.
+ * Builds a regular Dataflow pipeline with each of the mapped
+ * transforms, and makes sure that they work when the pipeline is
+ * executed in Spark.
+ */
+public class TransformTranslatorTest {
+
+  @Rule
+  public TestName name = new TestName();
+
+  private DirectPipelineRunner directRunner;
+  private SparkPipelineRunner sparkRunner;
+  private String testDataDirName;
+
+  @Before public void init() throws IOException {
+    sparkRunner = SparkPipelineRunner.create();
+    directRunner = DirectPipelineRunner.createForTest();
+    testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName())
+        + File.separator;
+    FileUtils.deleteDirectory(new File(testDataDirName));
+    new File(testDataDirName).mkdirs();
+  }
+
+  /**
+   * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline
+   * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
+   * transforms. Finally it makes sure that the results are the same for both runs.
+   */
+  @Test
+  public void testTextIOReadAndWriteTransforms() throws IOException {
+    String directOut = runPipeline("direct", directRunner);
+    String sparkOut = runPipeline("spark", sparkRunner);
+
+    List<String> directOutput =
+        Files.readAllLines(Paths.get(directOut + "-00000-of-00001"), Charsets.UTF_8);
+
+    List<String> sparkOutput =
+        Files.readAllLines(Paths.get(sparkOut + "-00000-of-00001"), Charsets.UTF_8);
+
+    // sort output to get a stable result (PCollections are not ordered)
+    Collections.sort(directOutput);
+    Collections.sort(sparkOutput);
+
+    Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray());
+  }
+
+  private String runPipeline(String name, PipelineRunner<?> runner) {
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out_" + name);
+    PCollection<String> lines =  p.apply(TextIO.Read.from("src/test/resources/test_text.txt"));
+    lines.apply(TextIO.Write.to(outFile));
+    runner.run(p);
+    return outFile;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
new file mode 100644
index 0000000..eb88542
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.joda.time.Duration;
+import org.junit.Test;
+
+public class WindowedWordCountTest {
+  private static final String[] WORDS_ARRAY = {
+          "hi there", "hi", "hi sue bob",
+          "hi sue", "", "bob hi"};
+  private static final Long[] TIMESTAMPS_ARRAY = {
+          60000L, 60000L, 60000L,
+          120000L, 120000L, 120000L};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+  private static final List<Long> TIMESTAMPS = Arrays.asList(TIMESTAMPS_ARRAY);
+  private static final List<String> EXPECTED_COUNT_SET =
+          ImmutableList.of("hi: 3", "there: 1", "sue: 1", "bob: 1",
+                  "hi: 2", "sue: 1", "bob: 1");
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    PCollection<String> inputWords = p.apply(Create.timestamped(WORDS, TIMESTAMPS))
+            .setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedWords = inputWords
+            .apply(Window.<String>into(FixedWindows.of(Duration.standardMinutes(1))));
+
+    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
new file mode 100644
index 0000000..ad7256c
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.coders;
+
+import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.junit.Test;
+
+/**
+ * Tests for WritableCoder.
+ */
+public class WritableCoderTest {
+
+  @Test
+  public void testIntWritableEncoding() throws Exception {
+    IntWritable value = new IntWritable(42);
+    WritableCoder<IntWritable> coder = WritableCoder.of(IntWritable.class);
+
+    CoderProperties.coderDecodeEncodeEqual(coder, value);
+  }
+
+  @Test
+  public void testNullWritableEncoding() throws Exception {
+    WritableCoder<NullWritable> coder = WritableCoder.of(NullWritable.class);
+
+    CoderProperties.coderDecodeEncodeEqual(coder, NullWritable.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
new file mode 100644
index 0000000..73dd2d3
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.Lists;
+import com.google.common.io.Resources;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.junit.Assert.assertEquals;
+
+public class AvroPipelineTest {
+
+  private File inputFile;
+  private File outputDir;
+
+  @Rule
+  public final TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    inputFile = tmpDir.newFile("test.avro");
+    outputDir = tmpDir.newFolder("out");
+    outputDir.delete();
+  }
+
+  @Test
+  public void testGeneric() throws Exception {
+    Schema schema = new Schema.Parser().parse(Resources.getResource("person.avsc").openStream());
+    GenericRecord savedRecord = new GenericData.Record(schema);
+    savedRecord.put("name", "John Doe");
+    savedRecord.put("age", 42);
+    savedRecord.put("siblingnames", Lists.newArrayList("Jimmy", "Jane"));
+    populateGenericFile(Lists.newArrayList(savedRecord), schema);
+
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    PCollection<GenericRecord> input = p.apply(
+        AvroIO.Read.from(inputFile.getAbsolutePath()).withSchema(schema));
+    input.apply(AvroIO.Write.to(outputDir.getAbsolutePath()).withSchema(schema));
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+
+    List<GenericRecord> records = readGenericFile();
+    assertEquals(Lists.newArrayList(savedRecord), records);
+  }
+
+  private void populateGenericFile(List<GenericRecord> genericRecords, Schema schema) throws IOException {
+    FileOutputStream outputStream = new FileOutputStream(this.inputFile);
+    GenericDatumWriter<GenericRecord> genericDatumWriter = new GenericDatumWriter<>(schema);
+
+    try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(genericDatumWriter)) {
+      dataFileWriter.create(schema, outputStream);
+      for (GenericRecord record : genericRecords) {
+        dataFileWriter.append(record);
+      }
+    }
+    outputStream.close();
+  }
+
+  private List<GenericRecord> readGenericFile() throws IOException {
+    List<GenericRecord> records = Lists.newArrayList();
+    GenericDatumReader<GenericRecord> genericDatumReader = new GenericDatumReader<>();
+    try (DataFileReader<GenericRecord> dataFileReader =
+             new DataFileReader<>(new File(outputDir + "-00000-of-00001"), genericDatumReader)) {
+      for (GenericRecord record : dataFileReader) {
+        records.add(record);
+      }
+    }
+    return records;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
new file mode 100644
index 0000000..39525b2
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import com.google.cloud.dataflow.examples.WordCount;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineOptionsFactory;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class NumShardsTest {
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+
+  private File outputDir;
+
+  @Rule
+  public final TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    outputDir = tmpDir.newFolder("out");
+    outputDir.delete();
+  }
+
+  @Test
+  public void testText() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(options);
+    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+    PCollection<String> output = inputWords.apply(new WordCount.CountWords())
+        .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+    output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt"));
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+
+    int count = 0;
+    Set<String> expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2");
+    for (File f : tmpDir.getRoot().listFiles(new FileFilter() {
+      @Override public boolean accept(File pathname) {
+        return pathname.getName().matches("out-.*\\.txt");
+      }
+    })) {
+      count++;
+      for (String line : Files.readLines(f, Charsets.UTF_8)) {
+        assertTrue(line + " not found", expected.remove(line));
+      }
+    }
+    assertEquals(3, count);
+    assertTrue(expected.isEmpty());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
new file mode 100644
index 0000000..7a9be8b
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class HadoopFileFormatPipelineTest {
+
+  private File inputFile;
+  private File outputFile;
+
+  @Rule
+  public final TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    inputFile = tmpDir.newFile("test.seq");
+    outputFile = tmpDir.newFolder("out");
+    outputFile.delete();
+  }
+
+  @Test
+  public void testSequenceFile() throws Exception {
+    populateFile();
+
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    @SuppressWarnings("unchecked")
+    Class<? extends FileInputFormat<IntWritable, Text>> inputFormatClass =
+        (Class<? extends FileInputFormat<IntWritable, Text>>) (Class<?>) SequenceFileInputFormat.class;
+    HadoopIO.Read.Bound<IntWritable,Text> read =
+        HadoopIO.Read.from(inputFile.getAbsolutePath(), inputFormatClass, IntWritable.class, Text.class);
+    PCollection<KV<IntWritable, Text>> input = p.apply(read);
+    @SuppressWarnings("unchecked")
+    Class<? extends FileOutputFormat<IntWritable, Text>> outputFormatClass =
+        (Class<? extends FileOutputFormat<IntWritable, Text>>) (Class<?>) TemplatedSequenceFileOutputFormat.class;
+    @SuppressWarnings("unchecked")
+    HadoopIO.Write.Bound<IntWritable,Text> write = HadoopIO.Write.to(outputFile.getAbsolutePath(),
+        outputFormatClass, IntWritable.class, Text.class);
+    input.apply(write.withoutSharding());
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+
+    IntWritable key = new IntWritable();
+    Text value = new Text();
+    try (Reader reader = new Reader(new Configuration(), Reader.file(new Path(outputFile.toURI())))) {
+      int i = 0;
+      while (reader.next(key, value)) {
+        assertEquals(i, key.get());
+        assertEquals("value-" + i, value.toString());
+        i++;
+      }
+    }
+  }
+
+  private void populateFile() throws IOException {
+    IntWritable key = new IntWritable();
+    Text value = new Text();
+    try (Writer writer = SequenceFile.createWriter(
+        new Configuration(),
+        Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class),
+        Writer.file(new Path(this.inputFile.toURI())))) {
+      for (int i = 0; i < 5; i++) {
+        key.set(i);
+        value.set("value-" + i);
+        writer.append(key, value);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
new file mode 100644
index 0000000..b1d35d5
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import org.junit.Test;
+
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber;
+import static org.junit.Assert.assertEquals;
+
+public class ShardNameBuilderTest {
+
+  @Test
+  public void testReplaceShardCount() {
+    assertEquals("", replaceShardCount("", 6));
+    assertEquals("-S-of-6", replaceShardCount("-S-of-N", 6));
+    assertEquals("-SS-of-06", replaceShardCount("-SS-of-NN", 6));
+    assertEquals("-S-of-60", replaceShardCount("-S-of-N", 60));
+    assertEquals("-SS-of-60", replaceShardCount("-SS-of-NN", 60));
+    assertEquals("/part-SSSSS", replaceShardCount("/part-SSSSS", 6));
+  }
+
+  @Test
+  public void testReplaceShardNumber() {
+    assertEquals("", replaceShardNumber("", 5));
+    assertEquals("-5-of-6", replaceShardNumber("-S-of-6", 5));
+    assertEquals("-05-of-06", replaceShardNumber("-SS-of-06", 5));
+    assertEquals("-59-of-60", replaceShardNumber("-S-of-60", 59));
+    assertEquals("-59-of-60", replaceShardNumber("-SS-of-60", 59));
+    assertEquals("/part-00005", replaceShardNumber("/part-SSSSS", 5));
+  }
+
+  @Test
+     public void testGetOutputDirectory() {
+    assertEquals("./", getOutputDirectory("foo", "-S-of-N"));
+    assertEquals("foo", getOutputDirectory("foo/bar", "-S-of-N"));
+    assertEquals("/foo", getOutputDirectory("/foo/bar", "-S-of-N"));
+    assertEquals("hdfs://foo/", getOutputDirectory("hdfs://foo/bar", "-S-of-N"));
+    assertEquals("foo/bar", getOutputDirectory("foo/bar", "/part-SSSSS"));
+    assertEquals("/foo/bar", getOutputDirectory("/foo/bar", "/part-SSSSS"));
+    assertEquals("hdfs://foo/bar", getOutputDirectory("hdfs://foo/bar", "/part-SSSSS"));
+  }
+
+  @Test
+  public void testGetOutputFilePrefix() {
+    assertEquals("foo", getOutputFilePrefix("foo", "-S-of-N"));
+    assertEquals("bar", getOutputFilePrefix("foo/bar", "-S-of-N"));
+    assertEquals("bar", getOutputFilePrefix("/foo/bar", "-S-of-N"));
+    assertEquals("bar", getOutputFilePrefix("hdfs://foo/bar", "-S-of-N"));
+    assertEquals("", getOutputFilePrefix("foo/bar", "/part-SSSSS"));
+    assertEquals("", getOutputFilePrefix("/foo/bar", "/part-SSSSS"));
+    assertEquals("", getOutputFilePrefix("hdfs://foo/bar", "/part-SSSSS"));
+  }
+
+  @Test
+  public void testGetOutputFileTemplate() {
+    assertEquals("-S-of-N", getOutputFileTemplate("foo", "-S-of-N"));
+    assertEquals("-S-of-N", getOutputFileTemplate("foo/bar", "-S-of-N"));
+    assertEquals("-S-of-N", getOutputFileTemplate("/foo/bar", "-S-of-N"));
+    assertEquals("-S-of-N", getOutputFileTemplate("hdfs://foo/bar", "-S-of-N"));
+    assertEquals("part-SSSSS", getOutputFileTemplate("foo/bar", "/part-SSSSS"));
+    assertEquals("part-SSSSS", getOutputFileTemplate("/foo/bar", "/part-SSSSS"));
+    assertEquals("part-SSSSS", getOutputFileTemplate("hdfs://foo/bar", "/part-SSSSS"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
new file mode 100644
index 0000000..828b26e
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
+
+import org.joda.time.Duration;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Test Flatten (union) implementation for streaming.
+ */
+public class FlattenStreamingTest {
+
+  private static final String[] WORDS_ARRAY_1 = {
+      "one", "two", "three", "four"};
+  private static final List<Iterable<String>> WORDS_QUEUE_1 =
+      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_1));
+  private static final String[] WORDS_ARRAY_2 = {
+          "five", "six", "seven", "eight"};
+  private static final List<Iterable<String>> WORDS_QUEUE_2 =
+          Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_2));
+  private static final String[] EXPECTED_UNION = {
+          "one", "two", "three", "four", "five", "six", "seven", "eight"};
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @Test
+  public void testRun() throws Exception {
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    PCollection<String> w1 =
+            p.apply(CreateStream.fromQueue(WORDS_QUEUE_1)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedW1 =
+            w1.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+    PCollection<String> w2 =
+            p.apply(CreateStream.fromQueue(WORDS_QUEUE_2)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedW2 =
+            w2.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+    PCollectionList<String> list = PCollectionList.of(windowedW1).and(windowedW2);
+    PCollection<String> union = list.apply(Flatten.<String>pCollections());
+
+    DataflowAssert.thatIterable(union.apply(View.<String>asIterable()))
+            .containsInAnyOrder(EXPECTED_UNION);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
new file mode 100644
index 0000000..e9e685b
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.io.KafkaIO;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
+import org.apache.beam.runners.spark.streaming.utils.EmbeddedKafkaCluster;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.joda.time.Duration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import kafka.serializer.StringDecoder;
+
+/**
+ * Test Kafka as input.
+ */
+public class KafkaStreamingTest {
+  private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
+          new EmbeddedKafkaCluster.EmbeddedZookeeper(17001);
+  private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER =
+          new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(),
+                  new Properties(), Collections.singletonList(6667));
+  private static final String TOPIC = "kafka_dataflow_test_topic";
+  private static final Map<String, String> KAFKA_MESSAGES = ImmutableMap.of(
+      "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
+  );
+  private static final Set<String> EXPECTED = ImmutableSet.of(
+      "k1,v1", "k2,v2", "k3,v3", "k4,v4"
+  );
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    EMBEDDED_ZOOKEEPER.startup();
+    EMBEDDED_KAFKA_CLUSTER.startup();
+
+    // write to Kafka
+    Properties producerProps = new Properties();
+    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
+    producerProps.put("request.required.acks", 1);
+    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
+    Serializer<String> stringSerializer = new StringSerializer();
+    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
+            new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
+      for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
+        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
+      }
+    }
+  }
+
+  @Test
+  public void testRun() throws Exception {
+    // test read from Kafka
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    Map<String, String> kafkaParams = ImmutableMap.of(
+            "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
+            "auto.offset.reset", "smallest"
+    );
+
+    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
+        StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
+        kafkaParams));
+    PCollection<KV<String, String>> windowedWords = kafkaInput
+        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
+
+    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn()));
+
+    DataflowAssert.thatIterable(formattedKV.apply(View.<String>asIterable()))
+        .containsInAnyOrder(EXPECTED);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    EMBEDDED_KAFKA_CLUSTER.shutdown();
+    EMBEDDED_ZOOKEEPER.shutdown();
+  }
+
+  private static class FormatKVFn extends DoFn<KV<String, String>, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().getKey() + "," + c.element().getValue());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
new file mode 100644
index 0000000..9a0609d
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SimpleWordCountTest;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
+
+import org.joda.time.Duration;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class SimpleStreamingWordCountTest {
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"};
+  private static final List<Iterable<String>> WORDS_QUEUE =
+      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY));
+  private static final Set<String> EXPECTED_COUNT_SET =
+      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
+
+  @Test
+  public void testRun() throws Exception {
+    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
+    options.setAppName(this.getClass().getSimpleName());
+    options.setRunner(SparkPipelineRunner.class);
+    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
+    Pipeline p = Pipeline.create(options);
+
+    PCollection<String> inputWords =
+        p.apply(CreateStream.fromQueue(WORDS_QUEUE)).setCoder(StringUtf8Coder.of());
+    PCollection<String> windowedWords = inputWords
+        .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
+
+    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
+
+    DataflowAssert.thatIterable(output.apply(View.<String>asIterable()))
+        .containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
+    res.close();
+
+    DataflowAssertStreaming.assertNoFailures(res);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
new file mode 100644
index 0000000..19759d7
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming.utils;
+
+import org.apache.beam.runners.spark.EvaluationResult;
+
+import org.junit.Assert;
+
+/**
+ * Since DataflowAssert doesn't propagate assert exceptions, use Aggregators to assert streaming
+ * success/failure counters.
+ */
+public final class DataflowAssertStreaming {
+  /**
+   * Copied aggregator names from {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert}
+   */
+  static final String SUCCESS_COUNTER = "DataflowAssertSuccess";
+  static final String FAILURE_COUNTER = "DataflowAssertFailure";
+
+  private DataflowAssertStreaming() {
+  }
+
+  public static void assertNoFailures(EvaluationResult res) {
+    int failures = res.getAggregatorValue(FAILURE_COUNTER, Integer.class);
+    Assert.assertEquals("Found " + failures + " failures, see the log for details", 0, failures);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
new file mode 100644
index 0000000..333453a
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
@@ -0,0 +1,314 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming.utils;
+
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * https://gist.github.com/fjavieralba/7930018
+ */
+public class EmbeddedKafkaCluster {
+
+  private static final Logger LOG = LoggerFactory.getLogger(EmbeddedKafkaCluster.class);
+
+  private final List<Integer> ports;
+  private final String zkConnection;
+  private final Properties baseProperties;
+
+  private final String brokerList;
+
+  private final List<KafkaServer> brokers;
+  private final List<File> logDirs;
+
+  public EmbeddedKafkaCluster(String zkConnection) {
+    this(zkConnection, new Properties());
+  }
+
+  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties) {
+    this(zkConnection, baseProperties, Collections.singletonList(-1));
+  }
+
+  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties, List<Integer> ports) {
+    this.zkConnection = zkConnection;
+    this.ports = resolvePorts(ports);
+    this.baseProperties = baseProperties;
+
+    this.brokers = new ArrayList<>();
+    this.logDirs = new ArrayList<>();
+
+    this.brokerList = constructBrokerList(this.ports);
+  }
+
+  private static List<Integer> resolvePorts(List<Integer> ports) {
+    List<Integer> resolvedPorts = new ArrayList<>();
+    for (Integer port : ports) {
+      resolvedPorts.add(resolvePort(port));
+    }
+    return resolvedPorts;
+  }
+
+  private static int resolvePort(int port) {
+    if (port == -1) {
+      return TestUtils.getAvailablePort();
+    }
+    return port;
+  }
+
+  private static String constructBrokerList(List<Integer> ports) {
+    StringBuilder sb = new StringBuilder();
+    for (Integer port : ports) {
+      if (sb.length() > 0) {
+        sb.append(",");
+      }
+      sb.append("localhost:").append(port);
+    }
+    return sb.toString();
+  }
+
+  public void startup() {
+    for (int i = 0; i < ports.size(); i++) {
+      Integer port = ports.get(i);
+      File logDir = TestUtils.constructTempDir("kafka-local");
+
+      Properties properties = new Properties();
+      properties.putAll(baseProperties);
+      properties.setProperty("zookeeper.connect", zkConnection);
+      properties.setProperty("broker.id", String.valueOf(i + 1));
+      properties.setProperty("host.name", "localhost");
+      properties.setProperty("port", Integer.toString(port));
+      properties.setProperty("log.dir", logDir.getAbsolutePath());
+      properties.setProperty("log.flush.interval.messages", String.valueOf(1));
+
+      KafkaServer broker = startBroker(properties);
+
+      brokers.add(broker);
+      logDirs.add(logDir);
+    }
+  }
+
+
+  private static KafkaServer startBroker(Properties props) {
+    KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
+    server.startup();
+    return server;
+  }
+
+  public Properties getProps() {
+    Properties props = new Properties();
+    props.putAll(baseProperties);
+    props.put("metadata.broker.list", brokerList);
+    props.put("zookeeper.connect", zkConnection);
+    return props;
+  }
+
+  public String getBrokerList() {
+    return brokerList;
+  }
+
+  public List<Integer> getPorts() {
+    return ports;
+  }
+
+  public String getZkConnection() {
+    return zkConnection;
+  }
+
+  public void shutdown() {
+    for (KafkaServer broker : brokers) {
+      try {
+        broker.shutdown();
+      } catch (Exception e) {
+        LOG.warn("{}", e.getMessage(), e);
+      }
+    }
+    for (File logDir : logDirs) {
+      try {
+        TestUtils.deleteFile(logDir);
+      } catch (FileNotFoundException e) {
+        LOG.warn("{}", e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "EmbeddedKafkaCluster{" + "brokerList='" + brokerList + "'}";
+  }
+
+  public static class EmbeddedZookeeper {
+    private int port = -1;
+    private int tickTime = 500;
+
+    private ServerCnxnFactory factory;
+    private File snapshotDir;
+    private File logDir;
+
+    public EmbeddedZookeeper() {
+      this(-1);
+    }
+
+    public EmbeddedZookeeper(int port) {
+      this(port, 500);
+    }
+
+    public EmbeddedZookeeper(int port, int tickTime) {
+      this.port = resolvePort(port);
+      this.tickTime = tickTime;
+    }
+
+    private static int resolvePort(int port) {
+      if (port == -1) {
+        return TestUtils.getAvailablePort();
+      }
+      return port;
+    }
+
+    public void startup() throws IOException {
+      if (this.port == -1) {
+        this.port = TestUtils.getAvailablePort();
+      }
+      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
+              1024);
+      this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
+      this.logDir = TestUtils.constructTempDir("embedded-zk/log");
+
+      try {
+        factory.startup(new ZooKeeperServer(snapshotDir, logDir, tickTime));
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+    }
+
+
+    public void shutdown() {
+      factory.shutdown();
+      try {
+        TestUtils.deleteFile(snapshotDir);
+      } catch (FileNotFoundException e) {
+        // ignore
+      }
+      try {
+        TestUtils.deleteFile(logDir);
+      } catch (FileNotFoundException e) {
+        // ignore
+      }
+    }
+
+    public String getConnection() {
+      return "localhost:" + port;
+    }
+
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    public void setTickTime(int tickTime) {
+      this.tickTime = tickTime;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    public int getTickTime() {
+      return tickTime;
+    }
+
+    @Override
+    public String toString() {
+      return "EmbeddedZookeeper{" + "connection=" + getConnection() + "}";
+    }
+  }
+
+  static class SystemTime implements Time {
+    @Override
+    public long milliseconds() {
+      return System.currentTimeMillis();
+    }
+
+    @Override
+    public long nanoseconds() {
+      return System.nanoTime();
+    }
+
+    @Override
+    public void sleep(long ms) {
+      try {
+        Thread.sleep(ms);
+      } catch (InterruptedException e) {
+        // Ignore
+      }
+    }
+  }
+
+  static final class TestUtils {
+    private static final Random RANDOM = new Random();
+
+    private TestUtils() {
+    }
+
+    static File constructTempDir(String dirPrefix) {
+      File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt
+              (10000000));
+      if (!file.mkdirs()) {
+        throw new RuntimeException("could not create temp directory: " + file.getAbsolutePath());
+      }
+      file.deleteOnExit();
+      return file;
+    }
+
+    static int getAvailablePort() {
+      try {
+        try (ServerSocket socket = new ServerSocket(0)) {
+          return socket.getLocalPort();
+        }
+      } catch (IOException e) {
+        throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e);
+      }
+    }
+
+    static boolean deleteFile(File path) throws FileNotFoundException {
+      if (!path.exists()) {
+        throw new FileNotFoundException(path.getAbsolutePath());
+      }
+      boolean ret = true;
+      if (path.isDirectory()) {
+        for (File f : path.listFiles()) {
+          ret = ret && deleteFile(f);
+        }
+      }
+      return ret && path.delete();
+    }
+  }
+}


[11/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
[BEAM-11] Spark runner directory structure and pom setup.


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

Branch: refs/heads/master
Commit: 41c4ca6ae284692bf3abd35491b4ed638b32c283
Parents: 46412e5
Author: Sela <an...@paypal.com>
Authored: Sat Mar 12 00:37:55 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:38:26 2016 +0200

----------------------------------------------------------------------
 runners/spark/pom.xml                           | 272 +++----
 .../com/cloudera/dataflow/hadoop/HadoopIO.java  | 202 -----
 .../dataflow/hadoop/NullWritableCoder.java      |  71 --
 .../cloudera/dataflow/hadoop/WritableCoder.java | 120 ---
 .../com/cloudera/dataflow/io/ConsoleIO.java     |  60 --
 .../com/cloudera/dataflow/io/CreateStream.java  |  66 --
 .../java/com/cloudera/dataflow/io/KafkaIO.java  | 128 ---
 .../dataflow/spark/BroadcastHelper.java         | 121 ---
 .../com/cloudera/dataflow/spark/ByteArray.java  |  52 --
 .../cloudera/dataflow/spark/CoderHelpers.java   | 185 -----
 .../cloudera/dataflow/spark/DoFnFunction.java   |  93 ---
 .../dataflow/spark/EvaluationContext.java       | 283 -------
 .../dataflow/spark/EvaluationResult.java        |  62 --
 .../dataflow/spark/MultiDoFnFunction.java       | 115 ---
 .../dataflow/spark/ShardNameBuilder.java        | 106 ---
 .../dataflow/spark/ShardNameTemplateAware.java  |  28 -
 .../dataflow/spark/ShardNameTemplateHelper.java |  58 --
 .../dataflow/spark/SparkContextFactory.java     |  66 --
 .../dataflow/spark/SparkPipelineEvaluator.java  |  52 --
 .../dataflow/spark/SparkPipelineOptions.java    |  39 -
 .../spark/SparkPipelineOptionsFactory.java      |  27 -
 .../spark/SparkPipelineOptionsRegistrar.java    |  27 -
 .../dataflow/spark/SparkPipelineRunner.java     | 252 ------
 .../spark/SparkPipelineRunnerRegistrar.java     |  27 -
 .../dataflow/spark/SparkPipelineTranslator.java |  27 -
 .../dataflow/spark/SparkProcessContext.java     | 250 ------
 .../dataflow/spark/SparkRuntimeContext.java     | 212 -----
 .../spark/TemplatedAvroKeyOutputFormat.java     |  40 -
 .../TemplatedSequenceFileOutputFormat.java      |  40 -
 .../spark/TemplatedTextOutputFormat.java        |  40 -
 .../dataflow/spark/TransformEvaluator.java      |  24 -
 .../dataflow/spark/TransformTranslator.java     | 800 ------------------
 .../dataflow/spark/WindowingHelpers.java        |  59 --
 .../spark/aggregators/AggAccumParam.java        |  35 -
 .../spark/aggregators/NamedAggregators.java     | 202 -----
 .../SparkStreamingPipelineOptions.java          |  40 -
 .../SparkStreamingPipelineOptionsFactory.java   |  27 -
 .../SparkStreamingPipelineOptionsRegistrar.java |  28 -
 .../streaming/StreamingEvaluationContext.java   | 226 ------
 .../streaming/StreamingTransformTranslator.java | 414 ----------
 .../StreamingWindowPipelineDetector.java        | 100 ---
 .../apache/beam/runners/spark/DoFnFunction.java |  94 +++
 .../beam/runners/spark/EvaluationContext.java   | 284 +++++++
 .../beam/runners/spark/EvaluationResult.java    |  62 ++
 .../beam/runners/spark/MultiDoFnFunction.java   | 116 +++
 .../beam/runners/spark/SparkContextFactory.java |  66 ++
 .../runners/spark/SparkPipelineEvaluator.java   |  52 ++
 .../runners/spark/SparkPipelineOptions.java     |  39 +
 .../spark/SparkPipelineOptionsFactory.java      |  27 +
 .../spark/SparkPipelineOptionsRegistrar.java    |  27 +
 .../beam/runners/spark/SparkPipelineRunner.java | 252 ++++++
 .../spark/SparkPipelineRunnerRegistrar.java     |  27 +
 .../runners/spark/SparkPipelineTranslator.java  |  27 +
 .../beam/runners/spark/SparkProcessContext.java | 257 ++++++
 .../beam/runners/spark/SparkRuntimeContext.java | 214 +++++
 .../beam/runners/spark/TransformEvaluator.java  |  24 +
 .../beam/runners/spark/TransformTranslator.java | 805 +++++++++++++++++++
 .../beam/runners/spark/WindowingHelpers.java    |  59 ++
 .../spark/aggregators/AggAccumParam.java        |  35 +
 .../spark/aggregators/NamedAggregators.java     | 202 +++++
 .../beam/runners/spark/coders/CoderHelpers.java | 186 +++++
 .../runners/spark/coders/NullWritableCoder.java |  71 ++
 .../runners/spark/coders/WritableCoder.java     | 120 +++
 .../apache/beam/runners/spark/io/ConsoleIO.java |  60 ++
 .../beam/runners/spark/io/CreateStream.java     |  66 ++
 .../apache/beam/runners/spark/io/KafkaIO.java   | 128 +++
 .../beam/runners/spark/io/hadoop/HadoopIO.java  | 200 +++++
 .../spark/io/hadoop/ShardNameBuilder.java       | 106 +++
 .../spark/io/hadoop/ShardNameTemplateAware.java |  28 +
 .../io/hadoop/ShardNameTemplateHelper.java      |  58 ++
 .../io/hadoop/TemplatedAvroKeyOutputFormat.java |  40 +
 .../TemplatedSequenceFileOutputFormat.java      |  40 +
 .../io/hadoop/TemplatedTextOutputFormat.java    |  40 +
 .../SparkStreamingPipelineOptions.java          |  40 +
 .../SparkStreamingPipelineOptionsFactory.java   |  27 +
 .../SparkStreamingPipelineOptionsRegistrar.java |  28 +
 .../streaming/StreamingEvaluationContext.java   | 226 ++++++
 .../streaming/StreamingTransformTranslator.java | 415 ++++++++++
 .../StreamingWindowPipelineDetector.java        | 101 +++
 .../runners/spark/util/BroadcastHelper.java     | 122 +++
 .../beam/runners/spark/util/ByteArray.java      |  52 ++
 ...ataflow.sdk.options.PipelineOptionsRegistrar |   4 +-
 ...dataflow.sdk.runners.PipelineRunnerRegistrar |   2 +-
 .../dataflow/hadoop/WritableCoderTest.java      |  42 -
 .../dataflow/spark/AvroPipelineTest.java        | 103 ---
 .../dataflow/spark/CombineGloballyTest.java     |  87 --
 .../dataflow/spark/CombinePerKeyTest.java       |  69 --
 .../com/cloudera/dataflow/spark/DeDupTest.java  |  55 --
 .../cloudera/dataflow/spark/DoFnOutputTest.java |  57 --
 .../cloudera/dataflow/spark/EmptyInputTest.java |  64 --
 .../spark/HadoopFileFormatPipelineTest.java     | 105 ---
 .../spark/MultiOutputWordCountTest.java         | 148 ----
 .../cloudera/dataflow/spark/NumShardsTest.java  |  89 --
 .../dataflow/spark/SerializationTest.java       | 183 -----
 .../dataflow/spark/ShardNameBuilderTest.java    |  82 --
 .../dataflow/spark/SideEffectsTest.java         |  77 --
 .../dataflow/spark/SimpleWordCountTest.java     | 117 ---
 .../spark/TestSparkPipelineOptionsFactory.java  |  34 -
 .../com/cloudera/dataflow/spark/TfIdfTest.java  |  60 --
 .../dataflow/spark/TransformTranslatorTest.java |  95 ---
 .../dataflow/spark/WindowedWordCountTest.java   |  63 --
 .../spark/streaming/FlattenStreamingTest.java   |  84 --
 .../spark/streaming/KafkaStreamingTest.java     | 133 ---
 .../streaming/SimpleStreamingWordCountTest.java |  73 --
 .../utils/DataflowAssertStreaming.java          |  39 -
 .../streaming/utils/EmbeddedKafkaCluster.java   | 314 --------
 .../beam/runners/spark/CombineGloballyTest.java |  88 ++
 .../beam/runners/spark/CombinePerKeyTest.java   |  65 ++
 .../apache/beam/runners/spark/DeDupTest.java    |  56 ++
 .../beam/runners/spark/DoFnOutputTest.java      |  58 ++
 .../beam/runners/spark/EmptyInputTest.java      |  65 ++
 .../runners/spark/MultiOutputWordCountTest.java | 132 +++
 .../beam/runners/spark/SerializationTest.java   | 177 ++++
 .../beam/runners/spark/SideEffectsTest.java     |  76 ++
 .../beam/runners/spark/SimpleWordCountTest.java | 111 +++
 .../spark/TestSparkPipelineOptionsFactory.java  |  34 +
 .../apache/beam/runners/spark/TfIdfTest.java    |  61 ++
 .../runners/spark/TransformTranslatorTest.java  |  95 +++
 .../runners/spark/WindowedWordCountTest.java    |  64 ++
 .../runners/spark/coders/WritableCoderTest.java |  42 +
 .../beam/runners/spark/io/AvroPipelineTest.java | 105 +++
 .../beam/runners/spark/io/NumShardsTest.java    |  93 +++
 .../io/hadoop/HadoopFileFormatPipelineTest.java | 107 +++
 .../spark/io/hadoop/ShardNameBuilderTest.java   |  82 ++
 .../spark/streaming/FlattenStreamingTest.java   |  84 ++
 .../spark/streaming/KafkaStreamingTest.java     | 133 +++
 .../streaming/SimpleStreamingWordCountTest.java |  73 ++
 .../utils/DataflowAssertStreaming.java          |  39 +
 .../streaming/utils/EmbeddedKafkaCluster.java   | 314 ++++++++
 129 files changed, 7113 insertions(+), 7119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 399e9e7..a060161 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -11,21 +11,136 @@ the specific language governing permissions and limitations under the
 License.
 -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
     <modelVersion>4.0.0</modelVersion>
-    <name>Dataflow on Spark</name>
-    <groupId>com.cloudera.dataflow.spark</groupId>
-    <artifactId>spark-dataflow</artifactId>
+
+    <parent>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>runners</artifactId>
+        <version>1.5.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>spark-runner</artifactId>
     <version>0.4.3-SNAPSHOT</version>
+
+    <name>Spark Beam Runner</name>
     <packaging>jar</packaging>
 
+    <inceptionYear>2014</inceptionYear>
+
+    <licenses>
+        <license>
+            <name>The Apache Software License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+        </license>
+    </licenses>
+
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <java.version>1.7</java.version>
         <spark.version>1.5.2</spark.version>
-        <google-cloud-dataflow-version>1.3.0</google-cloud-dataflow-version>
+        <beam.version>1.5.0-SNAPSHOT</beam.version>
     </properties>
 
+    <repositories>
+        <repository>
+            <id>apache.snapshots</id>
+            <name>Apache Development Snapshot Repository</name>
+            <url>https://repository.apache.org/content/repositories/snapshots/</url>
+            <releases>
+                <enabled>false</enabled>
+            </releases>
+            <snapshots>
+                <enabled>true</enabled>
+            </snapshots>
+        </repository>
+    </repositories>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_2.10</artifactId>
+            <version>${spark.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-streaming_2.10</artifactId>
+            <version>${spark.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-streaming-kafka_2.10</artifactId>
+            <version>${spark.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka_2.10</artifactId>
+            <version>0.8.2.1</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.cloud.dataflow</groupId>
+            <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
+            <version>${beam.version}</version>
+            <exclusions>
+                <!-- Use Hadoop/Spark's backend logger -->
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-jdk14</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.google.cloud.dataflow</groupId>
+            <artifactId>google-cloud-dataflow-java-examples-all</artifactId>
+            <version>${beam.version}</version>
+            <exclusions>
+                <!-- Use Hadoop/Spark's backend logger -->
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-jdk14</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro-mapred</artifactId>
+            <version>1.7.7</version>
+            <classifier>hadoop2</classifier>
+            <exclusions>
+                <!-- exclude old Jetty version of servlet API -->
+                <exclusion>
+                    <groupId>org.mortbay.jetty</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- test dependencies -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.12</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest-all</artifactId>
+            <version>1.3</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
     <build>
         <pluginManagement>
             <plugins>
@@ -231,20 +346,20 @@ License.
                             <goals>
                                 <goal>shade</goal>
                             </goals>
-                            <configuration>
-                                <relocations>
-                                    <!-- relocate Guava used by Dataflow (v18) since it conflicts with version used by Hadoop (v11) -->
-                                    <relocation>
-                                        <pattern>com.google.common</pattern>
-                                        <shadedPattern>com.cloudera.dataflow.spark.relocated.com.google.common</shadedPattern>
-                                    </relocation>
-                                </relocations>
-                                <shadedArtifactAttached>true</shadedArtifactAttached>
-                                <shadedClassifierName>spark-app</shadedClassifierName>
-                                <transformers>
-                                    <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                                </transformers>
-                            </configuration>
+                            <!--<configuration>-->
+                                <!--<relocations>-->
+                                    <!--&lt;!&ndash; relocate Guava used by Dataflow (v18) since it conflicts with version used by Hadoop (v11) &ndash;&gt;-->
+                                    <!--<relocation>-->
+                                        <!--<pattern>com.google.common</pattern>-->
+                                        <!--<shadedPattern>com.cloudera.dataflow.spark.relocated.com.google.common</shadedPattern>-->
+                                    <!--</relocation>-->
+                                <!--</relocations>-->
+                                <!--<shadedArtifactAttached>true</shadedArtifactAttached>-->
+                                <!--<shadedClassifierName>spark-app</shadedClassifierName>-->
+                                <!--<transformers>-->
+                                    <!--<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />-->
+                                <!--</transformers>-->
+                            <!--</configuration>-->
                         </execution>
                     </executions>
                 </plugin>
@@ -274,89 +389,6 @@ License.
         </plugins>
     </build>
 
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.spark</groupId>
-            <artifactId>spark-core_2.10</artifactId>
-            <version>${spark.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.spark</groupId>
-            <artifactId>spark-streaming_2.10</artifactId>
-            <version>${spark.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.spark</groupId>
-            <artifactId>spark-streaming-kafka_2.10</artifactId>
-            <version>${spark.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>kafka_2.10</artifactId>
-            <version>0.8.2.1</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-            <version>18.0</version>
-        </dependency>
-        <dependency>
-            <groupId>com.google.cloud.dataflow</groupId>
-            <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
-            <version>${google-cloud-dataflow-version}</version>
-            <exclusions>
-                <!-- Use Hadoop/Spark's backend logger -->
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-jdk14</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>com.google.cloud.dataflow</groupId>
-            <artifactId>google-cloud-dataflow-java-examples-all</artifactId>
-            <version>${google-cloud-dataflow-version}</version>
-            <exclusions>
-                <!-- Use Hadoop/Spark's backend logger -->
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-jdk14</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro-mapred</artifactId>
-            <version>1.7.7</version>
-            <classifier>hadoop2</classifier>
-            <exclusions>
-                <!-- exclude old Jetty version of servlet API -->
-                <exclusion>
-                    <groupId>org.mortbay.jetty</groupId>
-                    <artifactId>servlet-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <!-- test dependencies -->
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <version>4.12</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.hamcrest</groupId>
-            <artifactId>hamcrest-all</artifactId>
-            <version>1.3</version>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-
     <reporting>
         <plugins>
             <plugin>
@@ -380,25 +412,12 @@ License.
     </reporting>
 
 
-    <url>http://github.com/cloudera/spark-dataflow</url>
-    <inceptionYear>2014</inceptionYear>
-    <licenses>
-        <license>
-            <name>The Apache Software License, Version 2.0</name>
-            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-            <distribution>repo</distribution>
-        </license>
-    </licenses>
     <developers>
         <developer>
             <name>Cloudera, Inc.</name>
         </developer>
     </developers>
 
-    <issueManagement>
-        <system>GitHub</system>
-        <url>https://github.com/cloudera/spark-dataflow/issues</url>
-    </issueManagement>
     <scm>
         <connection>scm:git:https://github.com/cloudera/spark-dataflow.git</connection>
         <developerConnection>scm:git:https://github.com/cloudera/spark-dataflow.git</developerConnection>
@@ -410,31 +429,6 @@ License.
         <maven>3.2.1</maven>
     </prerequisites>
 
-    <repositories>
-        <repository>
-            <id>cloudera.repo</id>
-            <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
-            <name>Cloudera Repositories</name>
-            <releases>
-                <enabled>true</enabled>
-            </releases>
-            <snapshots>
-                <enabled>true</enabled>
-            </snapshots>
-        </repository>
-    </repositories>
-
-    <distributionManagement>
-        <repository>
-            <id>cloudera.repo</id>
-            <url>https://repository.cloudera.com/artifactory/libs-release-local</url>
-        </repository>
-        <snapshotRepository>
-            <id>cloudera.snapshots.repo</id>
-            <url>https://repository.cloudera.com/artifactory/libs-snapshot-local</url>
-        </snapshotRepository>
-    </distributionManagement>
-
     <profiles>
         <profile>
             <id>release-sign-artifacts</id>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java
deleted file mode 100644
index c79f211..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.hadoop;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-
-import com.cloudera.dataflow.spark.ShardNameTemplateAware;
-
-public final class HadoopIO {
-
-  private HadoopIO() {
-  }
-
-  public static final class Read {
-
-    private Read() {
-    }
-
-    public static <K, V> Bound<K, V> from(String filepattern,
-        Class<? extends FileInputFormat<K, V>> format, Class<K> key, Class<V> value) {
-      return new Bound<>(filepattern, format, key, value);
-    }
-
-    public static class Bound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
-
-      private final String filepattern;
-      private final Class<? extends FileInputFormat<K, V>> formatClass;
-      private final Class<K> keyClass;
-      private final Class<V> valueClass;
-
-      Bound(String filepattern, Class<? extends FileInputFormat<K, V>> format, Class<K> key,
-          Class<V> value) {
-        Preconditions.checkNotNull(filepattern,
-                                   "need to set the filepattern of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(format,
-                                   "need to set the format class of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(key,
-                                   "need to set the key class of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(value,
-                                   "need to set the value class of an HadoopIO.Read transform");
-        this.filepattern = filepattern;
-        this.formatClass = format;
-        this.keyClass = key;
-        this.valueClass = value;
-      }
-
-      public String getFilepattern() {
-        return filepattern;
-      }
-
-      public Class<? extends FileInputFormat<K, V>> getFormatClass() {
-        return formatClass;
-      }
-
-      public Class<V> getValueClass() {
-        return valueClass;
-      }
-
-      public Class<K> getKeyClass() {
-        return keyClass;
-      }
-
-      @Override
-      public PCollection<KV<K, V>> apply(PInput input) {
-        return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
-            WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED);
-      }
-
-    }
-
-  }
-
-  public static final class Write {
-
-    private Write() {
-    }
-
-    public static <K, V> Bound<K, V> to(String filenamePrefix,
-        Class<? extends FileOutputFormat<K, V>> format, Class<K> key, Class<V> value) {
-      return new Bound<>(filenamePrefix, format, key, value);
-    }
-
-    public static class Bound<K, V> extends PTransform<PCollection<KV<K, V>>, PDone> {
-
-      /** The filename to write to. */
-      private final String filenamePrefix;
-      /** Suffix to use for each filename. */
-      private final String filenameSuffix;
-      /** Requested number of shards.  0 for automatic. */
-      private final int numShards;
-      /** Shard template string. */
-      private final String shardTemplate;
-      private final Class<? extends FileOutputFormat<K, V>> formatClass;
-      private final Class<K> keyClass;
-      private final Class<V> valueClass;
-      private final Map<String, String> configurationProperties;
-
-      Bound(String filenamePrefix, Class<? extends FileOutputFormat<K, V>> format,
-          Class<K> key,
-          Class<V> value) {
-        this(filenamePrefix, "", 0, ShardNameTemplate.INDEX_OF_MAX, format, key, value,
-            new HashMap<String, String>());
-      }
-
-      Bound(String filenamePrefix, String filenameSuffix, int numShards,
-          String shardTemplate, Class<? extends FileOutputFormat<K, V>> format,
-          Class<K> key, Class<V> value, Map<String, String> configurationProperties) {
-        this.filenamePrefix = filenamePrefix;
-        this.filenameSuffix = filenameSuffix;
-        this.numShards = numShards;
-        this.shardTemplate = shardTemplate;
-        this.formatClass = format;
-        this.keyClass = key;
-        this.valueClass = value;
-        this.configurationProperties = configurationProperties;
-      }
-
-      public Bound<K, V> withoutSharding() {
-        return new Bound<>(filenamePrefix, filenameSuffix, 1, "", formatClass,
-            keyClass, valueClass, configurationProperties);
-      }
-
-      public Bound<K, V> withConfigurationProperty(String key, String value) {
-        configurationProperties.put(key, value);
-        return this;
-      }
-
-      public String getFilenamePrefix() {
-        return filenamePrefix;
-      }
-
-      public String getShardTemplate() {
-        return shardTemplate;
-      }
-
-      public int getNumShards() {
-        return numShards;
-      }
-
-      public String getFilenameSuffix() {
-        return filenameSuffix;
-      }
-
-      public Class<? extends FileOutputFormat<K, V>> getFormatClass() {
-        return formatClass;
-      }
-
-      public Class<V> getValueClass() {
-        return valueClass;
-      }
-
-      public Class<K> getKeyClass() {
-        return keyClass;
-      }
-
-      public Map<String, String> getConfigurationProperties() {
-        return configurationProperties;
-      }
-
-      @Override
-      public PDone apply(PCollection<KV<K, V>> input) {
-        Preconditions.checkNotNull(filenamePrefix,
-            "need to set the filename prefix of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(formatClass,
-            "need to set the format class of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(keyClass,
-            "need to set the key class of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(valueClass,
-            "need to set the value class of an HadoopIO.Write transform");
-
-        Preconditions.checkArgument(ShardNameTemplateAware.class.isAssignableFrom(formatClass),
-            "Format class must implement " + ShardNameTemplateAware.class.getName());
-
-        return PDone.in(input.getPipeline());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/NullWritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/NullWritableCoder.java b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/NullWritableCoder.java
deleted file mode 100644
index 5e5d391..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/NullWritableCoder.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.hadoop;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import org.apache.hadoop.io.NullWritable;
-
-public final class NullWritableCoder extends WritableCoder<NullWritable> {
-  private static final long serialVersionUID = 1L;
-
-  @JsonCreator
-  public static NullWritableCoder of() {
-    return INSTANCE;
-  }
-
-  private static final NullWritableCoder INSTANCE = new NullWritableCoder();
-
-  private NullWritableCoder() {
-    super(NullWritable.class);
-  }
-
-  @Override
-  public void encode(NullWritable value, OutputStream outStream, Context context) {
-    // nothing to write
-  }
-
-  @Override
-  public NullWritable decode(InputStream inStream, Context context) {
-    return NullWritable.get();
-  }
-
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * Returns true since registerByteSizeObserver() runs in constant time.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(NullWritable value, Context context) {
-    return true;
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(NullWritable value, Context context) {
-    return 0;
-  }
-
-  @Override
-  public void verifyDeterministic() throws Coder.NonDeterministicException {
-    // NullWritableCoder is deterministic
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java
deleted file mode 100644
index 324b203..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.hadoop;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.util.List;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-
-/**
- * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}.
- *
- * <p> To use, specify the coder type on a PCollection:
- * <pre>
- * {@code
- *   PCollection<MyRecord> records =
- *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
- * }
- * </pre>
- *
- * @param <T> the type of elements handled by this coder
- */
-public class WritableCoder<T extends Writable> extends StandardCoder<T> {
-  private static final long serialVersionUID = 0L;
-
-  /**
-   * Returns a {@code WritableCoder} instance for the provided element class.
-   * @param <T> the element type
-   * @param clazz the element class
-   * @return a {@code WritableCoder} instance for the provided element class
-   */
-  public static <T extends Writable> WritableCoder<T> of(Class<T> clazz) {
-    if (clazz.equals(NullWritable.class)) {
-      @SuppressWarnings("unchecked")
-      WritableCoder<T> result = (WritableCoder<T>) NullWritableCoder.of();
-      return result;
-    }
-    return new WritableCoder<>(clazz);
-  }
-
-  @JsonCreator
-  @SuppressWarnings("unchecked")
-  public static WritableCoder<?> of(@JsonProperty("type") String classType)
-      throws ClassNotFoundException {
-    Class<?> clazz = Class.forName(classType);
-    if (!Writable.class.isAssignableFrom(clazz)) {
-      throw new ClassNotFoundException(
-          "Class " + classType + " does not implement Writable");
-    }
-    return of((Class<? extends Writable>) clazz);
-  }
-
-  private final Class<T> type;
-
-  public WritableCoder(Class<T> type) {
-    this.type = type;
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context) throws IOException {
-    value.write(new DataOutputStream(outStream));
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws IOException {
-    try {
-      T t = type.getConstructor().newInstance();
-      t.readFields(new DataInputStream(inStream));
-      return t;
-    } catch (NoSuchMethodException | InstantiationException | IllegalAccessException e) {
-      throw new CoderException("unable to deserialize record", e);
-    } catch (InvocationTargetException ite) {
-      throw new CoderException("unable to deserialize record", ite.getCause());
-    }
-  }
-
-  @Override
-  public List<Coder<?>> getCoderArguments() {
-    return null;
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    result.put("type", type.getName());
-    return result;
-  }
-
-  @Override
-  public void verifyDeterministic() throws Coder.NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Hadoop Writable may be non-deterministic.");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/io/ConsoleIO.java b/runners/spark/src/main/java/com/cloudera/dataflow/io/ConsoleIO.java
deleted file mode 100644
index bc19b39..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/io/ConsoleIO.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.io;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-/**
- * Print to console.
- */
-public final class ConsoleIO {
-
-  private ConsoleIO() {
-  }
-
-  public static final class Write {
-
-    private Write() {
-    }
-
-    public static <T> Unbound<T> from() {
-      return new Unbound<>(10);
-    }
-
-    public static <T> Unbound<T> from(int num) {
-      return new Unbound<>(num);
-    }
-
-    public static class Unbound<T> extends PTransform<PCollection<T>, PDone> {
-
-      private final int num;
-
-      Unbound(int num) {
-        this.num = num;
-      }
-
-      public int getNum() {
-        return num;
-      }
-
-      @Override
-      public PDone apply(PCollection<T> input) {
-        return PDone.in(input.getPipeline());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/io/CreateStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/io/CreateStream.java b/runners/spark/src/main/java/com/cloudera/dataflow/io/CreateStream.java
deleted file mode 100644
index 9a99278..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/io/CreateStream.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.io;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.base.Preconditions;
-
-/**
- * Create an input stream from Queue.
- *
- * @param <T> stream type
- */
-public final class CreateStream<T> {
-
-  private CreateStream() {
-  }
-
-  /**
-   * Define the input stream to create from queue.
-   *
-   * @param queuedValues  defines the input stream
-   * @param <T>           stream type
-   * @return the queue that defines the input stream
-   */
-  public static <T> QueuedValues<T> fromQueue(Iterable<Iterable<T>> queuedValues) {
-    return new QueuedValues<>(queuedValues);
-  }
-
-  public static final class QueuedValues<T> extends PTransform<PInput, PCollection<T>> {
-
-    private final Iterable<Iterable<T>> queuedValues;
-
-    QueuedValues(Iterable<Iterable<T>> queuedValues) {
-      Preconditions.checkNotNull(queuedValues,
-              "need to set the queuedValues of an Create.QueuedValues transform");
-      this.queuedValues = queuedValues;
-    }
-
-    public Iterable<Iterable<T>> getQueuedValues() {
-      return queuedValues;
-    }
-
-    @Override
-    public PCollection<T> apply(PInput input) {
-      // Spark streaming micro batches are bounded by default
-      return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
-          WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/io/KafkaIO.java b/runners/spark/src/main/java/com/cloudera/dataflow/io/KafkaIO.java
deleted file mode 100644
index 154e6da..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/io/KafkaIO.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.io;
-
-import java.util.Map;
-import java.util.Set;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.base.Preconditions;
-
-import kafka.serializer.Decoder;
-
-/**
- * Read stream from Kafka.
- */
-public final class KafkaIO {
-
-  private KafkaIO() {
-  }
-
-  public static final class Read {
-
-    private Read() {
-    }
-
-    /**
-     * Define the Kafka consumption.
-     *
-     * @param keyDecoder    {@link Decoder} to decode the Kafka message key
-     * @param valueDecoder  {@link Decoder} to decode the Kafka message value
-     * @param key           Kafka message key Class
-     * @param value         Kafka message value Class
-     * @param topics        Kafka topics to subscribe
-     * @param kafkaParams   map of Kafka parameters
-     * @param <K>           Kafka message key Class type
-     * @param <V>           Kafka message value Class type
-     * @return KafkaIO Unbound input
-     */
-    public static <K, V> Unbound<K, V> from(Class<? extends Decoder<K>> keyDecoder,
-                                            Class<? extends Decoder<V>> valueDecoder,
-                                            Class<K> key,
-                                            Class<V> value, Set<String> topics,
-                                            Map<String, String> kafkaParams) {
-      return new Unbound<>(keyDecoder, valueDecoder, key, value, topics, kafkaParams);
-    }
-
-    public static class Unbound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
-
-      private final Class<? extends Decoder<K>> keyDecoderClass;
-      private final Class<? extends Decoder<V>> valueDecoderClass;
-      private final Class<K> keyClass;
-      private final Class<V> valueClass;
-      private final Set<String> topics;
-      private final Map<String, String> kafkaParams;
-
-      Unbound(Class<? extends Decoder<K>> keyDecoder,
-              Class<? extends Decoder<V>> valueDecoder, Class<K> key,
-              Class<V> value, Set<String> topics, Map<String, String> kafkaParams) {
-        Preconditions.checkNotNull(keyDecoder,
-            "need to set the key decoder class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(valueDecoder,
-            "need to set the value decoder class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(key,
-            "need to set the key class of aKafkaIO.Read transform");
-        Preconditions.checkNotNull(value,
-            "need to set the value class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(topics,
-            "need to set the topics of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(kafkaParams,
-            "need to set the kafkaParams of a KafkaIO.Read transform");
-        this.keyDecoderClass = keyDecoder;
-        this.valueDecoderClass = valueDecoder;
-        this.keyClass = key;
-        this.valueClass = value;
-        this.topics = topics;
-        this.kafkaParams = kafkaParams;
-      }
-
-      public Class<? extends Decoder<K>> getKeyDecoderClass() {
-        return keyDecoderClass;
-      }
-
-      public Class<? extends Decoder<V>> getValueDecoderClass() {
-        return valueDecoderClass;
-      }
-
-      public Class<V> getValueClass() {
-        return valueClass;
-      }
-
-      public Class<K> getKeyClass() {
-        return keyClass;
-      }
-
-      public Set<String> getTopics() {
-        return topics;
-      }
-
-      public Map<String, String> getKafkaParams() {
-        return kafkaParams;
-      }
-
-      @Override
-      public PCollection<KV<K, V>> apply(PInput input) {
-        // Spark streaming micro batches are bounded by default
-        return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
-            WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java
deleted file mode 100644
index 8dca939..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.Serializable;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.broadcast.Broadcast;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class BroadcastHelper<T> implements Serializable {
-
-  /**
-   * If the property {@code dataflow.spark.directBroadcast} is set to
-   * {@code true} then Spark serialization (Kryo) will be used to broadcast values
-   * in View objects. By default this property is not set, and values are coded using
-   * the appropriate {@link Coder}.
-   */
-  public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast";
-
-  private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class);
-
-  public static <T> BroadcastHelper<T> create(T value, Coder<T> coder) {
-    if (Boolean.parseBoolean(System.getProperty(DIRECT_BROADCAST, "false"))) {
-      return new DirectBroadcastHelper<>(value);
-    }
-    return new CodedBroadcastHelper<>(value, coder);
-  }
-
-  public abstract T getValue();
-
-  public abstract void broadcast(JavaSparkContext jsc);
-
-  /**
-   * A {@link com.cloudera.dataflow.spark.BroadcastHelper} that relies on the underlying
-   * Spark serialization (Kryo) to broadcast values. This is appropriate when
-   * broadcasting very large values, since no copy of the object is made.
-   * @param <T>
-   */
-  static class DirectBroadcastHelper<T> extends BroadcastHelper<T> {
-    private Broadcast<T> bcast;
-    private transient T value;
-
-    DirectBroadcastHelper(T value) {
-      this.value = value;
-    }
-
-    @Override
-    public synchronized T getValue() {
-      if (value == null) {
-        value = bcast.getValue();
-      }
-      return value;
-    }
-
-    @Override
-    public void broadcast(JavaSparkContext jsc) {
-      this.bcast = jsc.broadcast(value);
-    }
-  }
-
-  /**
-   * A {@link com.cloudera.dataflow.spark.BroadcastHelper} that uses a
-   * {@link Coder} to encode values as byte arrays
-   * before broadcasting.
-   * @param <T>
-   */
-  static class CodedBroadcastHelper<T> extends BroadcastHelper<T> {
-    private Broadcast<byte[]> bcast;
-    private final Coder<T> coder;
-    private transient T value;
-
-    CodedBroadcastHelper(T value, Coder<T> coder) {
-      this.value = value;
-      this.coder = coder;
-    }
-
-    @Override
-    public synchronized T getValue() {
-      if (value == null) {
-        value = deserialize();
-      }
-      return value;
-    }
-
-    @Override
-    public void broadcast(JavaSparkContext jsc) {
-      this.bcast = jsc.broadcast(CoderHelpers.toByteArray(value, coder));
-    }
-
-    private T deserialize() {
-      T val;
-      try {
-        val = coder.decode(new ByteArrayInputStream(bcast.value()),
-            new Coder.Context(true));
-      } catch (IOException ioe) {
-        // this should not ever happen, log it if it does.
-        LOG.warn(ioe.getMessage());
-        val = null;
-      }
-      return val;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java
deleted file mode 100644
index 06db572..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-import com.google.common.primitives.UnsignedBytes;
-
-class ByteArray implements Serializable, Comparable<ByteArray> {
-
-  private final byte[] value;
-
-  ByteArray(byte[] value) {
-    this.value = value;
-  }
-
-  public byte[] getValue() {
-    return value;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    ByteArray byteArray = (ByteArray) o;
-    return Arrays.equals(value, byteArray.value);
-  }
-
-  @Override
-  public int hashCode() {
-    return value != null ? Arrays.hashCode(value) : 0;
-  }
-
-  @Override
-  public int compareTo(ByteArray other) {
-    return UnsignedBytes.lexicographicalComparator().compare(value, other.value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java
deleted file mode 100644
index 0ae06c1..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.common.collect.Iterables;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.PairFunction;
-import scala.Tuple2;
-
-/**
- * Serialization utility class.
- */
-public final class CoderHelpers {
-  private CoderHelpers() {
-  }
-
-  /**
-   * Utility method for serializing an object using the specified coder.
-   *
-   * @param value Value to serialize.
-   * @param coder Coder to serialize with.
-   * @param <T> type of value that is serialized
-   * @return Byte array representing serialized object.
-   */
-  static <T> byte[] toByteArray(T value, Coder<T> coder) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    try {
-      coder.encode(value, baos, new Coder.Context(true));
-    } catch (IOException e) {
-      throw new IllegalStateException("Error encoding value: " + value, e);
-    }
-    return baos.toByteArray();
-  }
-
-  /**
-   * Utility method for serializing a Iterable of values using the specified coder.
-   *
-   * @param values Values to serialize.
-   * @param coder  Coder to serialize with.
-   * @param <T> type of value that is serialized
-   * @return List of bytes representing serialized objects.
-   */
-  static <T> List<byte[]> toByteArrays(Iterable<T> values, Coder<T> coder) {
-    List<byte[]> res = new LinkedList<>();
-    for (T value : values) {
-      res.add(toByteArray(value, coder));
-    }
-    return res;
-  }
-
-  /**
-   * Utility method for deserializing a byte array using the specified coder.
-   *
-   * @param serialized bytearray to be deserialized.
-   * @param coder      Coder to deserialize with.
-   * @param <T>        Type of object to be returned.
-   * @return Deserialized object.
-   */
-  static <T> T fromByteArray(byte[] serialized, Coder<T> coder) {
-    ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
-    try {
-      return coder.decode(bais, new Coder.Context(true));
-    } catch (IOException e) {
-      throw new IllegalStateException("Error decoding bytes for coder: " + coder, e);
-    }
-  }
-
-  /**
-   * A function wrapper for converting an object to a bytearray.
-   *
-   * @param coder Coder to serialize with.
-   * @param <T>   The type of the object being serialized.
-   * @return A function that accepts an object and returns its coder-serialized form.
-   */
-  static <T> Function<T, byte[]> toByteFunction(final Coder<T> coder) {
-    return new Function<T, byte[]>() {
-      @Override
-      public byte[] call(T t) throws Exception {
-        return toByteArray(t, coder);
-      }
-    };
-  }
-
-  /**
-   * A function wrapper for converting a byte array to an object.
-   *
-   * @param coder Coder to deserialize with.
-   * @param <T>   The type of the object being deserialized.
-   * @return A function that accepts a byte array and returns its corresponding object.
-   */
-  static <T> Function<byte[], T> fromByteFunction(final Coder<T> coder) {
-    return new Function<byte[], T>() {
-      @Override
-      public T call(byte[] bytes) throws Exception {
-        return fromByteArray(bytes, coder);
-      }
-    };
-  }
-
-  /**
-   * A function wrapper for converting a key-value pair to a byte array pair.
-   *
-   * @param keyCoder Coder to serialize keys.
-   * @param valueCoder Coder to serialize values.
-   * @param <K>   The type of the key being serialized.
-   * @param <V>   The type of the value being serialized.
-   * @return A function that accepts a key-value pair and returns a pair of byte arrays.
-   */
-  static <K, V> PairFunction<Tuple2<K, V>, ByteArray, byte[]> toByteFunction(
-      final Coder<K> keyCoder, final Coder<V> valueCoder) {
-    return new PairFunction<Tuple2<K, V>, ByteArray, byte[]>() {
-      @Override
-      public Tuple2<ByteArray, byte[]> call(Tuple2<K, V> kv) {
-        return new Tuple2<>(new ByteArray(toByteArray(kv._1(), keyCoder)), toByteArray(kv._2(),
-            valueCoder));
-      }
-    };
-  }
-
-  /**
-   * A function wrapper for converting a byte array pair to a key-value pair.
-   *
-   * @param keyCoder Coder to deserialize keys.
-   * @param valueCoder Coder to deserialize values.
-   * @param <K>   The type of the key being deserialized.
-   * @param <V>   The type of the value being deserialized.
-   * @return A function that accepts a pair of byte arrays and returns a key-value pair.
-   */
-  static <K, V> PairFunction<Tuple2<ByteArray, byte[]>, K, V> fromByteFunction(
-      final Coder<K> keyCoder, final Coder<V> valueCoder) {
-    return new PairFunction<Tuple2<ByteArray, byte[]>, K, V>() {
-      @Override
-      public Tuple2<K, V> call(Tuple2<ByteArray, byte[]> tuple) {
-        return new Tuple2<>(fromByteArray(tuple._1().getValue(), keyCoder),
-            fromByteArray(tuple._2(), valueCoder));
-      }
-    };
-  }
-
-  /**
-   * A function wrapper for converting a byte array pair to a key-value pair, where
-   * values are {@link Iterable}.
-   *
-   * @param keyCoder Coder to deserialize keys.
-   * @param valueCoder Coder to deserialize values.
-   * @param <K>   The type of the key being deserialized.
-   * @param <V>   The type of the value being deserialized.
-   * @return A function that accepts a pair of byte arrays and returns a key-value pair.
-   */
-  static <K, V> PairFunction<Tuple2<ByteArray, Iterable<byte[]>>, K, Iterable<V>>
-      fromByteFunctionIterable(final Coder<K> keyCoder, final Coder<V> valueCoder) {
-    return new PairFunction<Tuple2<ByteArray, Iterable<byte[]>>, K, Iterable<V>>() {
-      @Override
-      public Tuple2<K, Iterable<V>> call(Tuple2<ByteArray, Iterable<byte[]>> tuple) {
-        return new Tuple2<>(fromByteArray(tuple._1().getValue(), keyCoder),
-          Iterables.transform(tuple._2(), new com.google.common.base.Function<byte[], V>() {
-            @Override
-            public V apply(byte[] bytes) {
-              return fromByteArray(bytes, valueCoder);
-            }
-          }));
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java
deleted file mode 100644
index 2bcfec3..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.apache.spark.api.java.function.FlatMapFunction;
-
-/**
- * Dataflow's Do functions correspond to Spark's FlatMap functions.
- *
- * @param <I> Input element type.
- * @param <O> Output element type.
- */
-public class DoFnFunction<I, O> implements FlatMapFunction<Iterator<WindowedValue<I>>,
-    WindowedValue<O>> {
-  private final DoFn<I, O> mFunction;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  /**
-   * @param fn         DoFunction to be wrapped.
-   * @param runtime    Runtime to apply function in.
-   * @param sideInputs Side inputs used in DoFunction.
-   */
-  public DoFnFunction(DoFn<I, O> fn,
-               SparkRuntimeContext runtime,
-               Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    this.mFunction = fn;
-    this.mRuntimeContext = runtime;
-    this.mSideInputs = sideInputs;
-  }
-
-  @Override
-  public Iterable<WindowedValue<O>> call(Iterator<WindowedValue<I>> iter) throws
-      Exception {
-    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
-    ctxt.setup();
-    mFunction.startBundle(ctxt);
-    return ctxt.getOutputIterable(iter, mFunction);
-  }
-
-  private class ProcCtxt extends SparkProcessContext<I, O, WindowedValue<O>> {
-
-    private final List<WindowedValue<O>> outputs = new LinkedList<>();
-
-    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
-        BroadcastHelper<?>> sideInputs) {
-      super(fn, runtimeContext, sideInputs);
-    }
-
-    @Override
-    public synchronized void output(O o) {
-      outputs.add(windowedValue != null ? windowedValue.withValue(o) :
-          WindowedValue.valueInEmptyWindows(o));
-    }
-
-    @Override
-    public synchronized void output(WindowedValue<O> o) {
-      outputs.add(o);
-    }
-
-    @Override
-    protected void clearOutput() {
-      outputs.clear();
-    }
-
-    @Override
-    protected Iterator<WindowedValue<O>> getOutputIterator() {
-      return outputs.iterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
deleted file mode 100644
index a6ac6c2..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-
-
-/**
- * Evaluation context allows us to define how pipeline instructions.
- */
-public class EvaluationContext implements EvaluationResult {
-  private final JavaSparkContext jsc;
-  private final Pipeline pipeline;
-  private final SparkRuntimeContext runtime;
-  private final Map<PValue, RDDHolder<?>> pcollections = new LinkedHashMap<>();
-  private final Set<RDDHolder<?>> leafRdds = new LinkedHashSet<>();
-  private final Set<PValue> multireads = new LinkedHashSet<>();
-  private final Map<PValue, Object> pobjects = new LinkedHashMap<>();
-  private final Map<PValue, Iterable<? extends WindowedValue<?>>> pview = new LinkedHashMap<>();
-  protected AppliedPTransform<?, ?, ?> currentTransform;
-
-  public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
-    this.jsc = jsc;
-    this.pipeline = pipeline;
-    this.runtime = new SparkRuntimeContext(jsc, pipeline);
-  }
-
-  /**
-   * Holds an RDD or values for deferred conversion to an RDD if needed. PCollections are
-   * sometimes created from a collection of objects (using RDD parallelize) and then
-   * only used to create View objects; in which case they do not need to be
-   * converted to bytes since they are not transferred across the network until they are
-   * broadcast.
-   */
-  private class RDDHolder<T> {
-
-    private Iterable<T> values;
-    private Coder<T> coder;
-    private JavaRDDLike<WindowedValue<T>, ?> rdd;
-
-    RDDHolder(Iterable<T> values, Coder<T> coder) {
-      this.values = values;
-      this.coder = coder;
-    }
-
-    RDDHolder(JavaRDDLike<WindowedValue<T>, ?> rdd) {
-      this.rdd = rdd;
-    }
-
-    JavaRDDLike<WindowedValue<T>, ?> getRDD() {
-      if (rdd == null) {
-        Iterable<WindowedValue<T>> windowedValues = Iterables.transform(values,
-            new Function<T, WindowedValue<T>>() {
-            @Override
-            public WindowedValue<T> apply(T t) {
-             // TODO: this is wrong if T is a TimestampedValue
-              return WindowedValue.valueInEmptyWindows(t);
-            }
-        });
-        WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder =
-            WindowedValue.getValueOnlyCoder(coder);
-        rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
-            .map(CoderHelpers.fromByteFunction(windowCoder));
-      }
-      return rdd;
-    }
-
-    Iterable<T> getValues(PCollection<T> pcollection) {
-      if (values == null) {
-        coder = pcollection.getCoder();
-        JavaRDDLike<byte[], ?> bytesRDD = rdd.map(WindowingHelpers.<T>unwindowFunction())
-            .map(CoderHelpers.toByteFunction(coder));
-        List<byte[]> clientBytes = bytesRDD.collect();
-        values = Iterables.transform(clientBytes, new Function<byte[], T>() {
-          @Override
-          public T apply(byte[] bytes) {
-            return CoderHelpers.fromByteArray(bytes, coder);
-          }
-        });
-      }
-      return values;
-    }
-
-    Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
-      return Iterables.transform(get(pcollection), new Function<T, WindowedValue<T>>() {
-        @Override
-        public WindowedValue<T> apply(T t) {
-          return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place?
-        }
-      });
-    }
-  }
-
-  protected JavaSparkContext getSparkContext() {
-    return jsc;
-  }
-
-  protected Pipeline getPipeline() {
-    return pipeline;
-  }
-
-  protected SparkRuntimeContext getRuntimeContext() {
-    return runtime;
-  }
-
-  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
-    this.currentTransform = transform;
-  }
-
-  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
-    return currentTransform;
-  }
-
-  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
-    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-        "can only be called with current transform");
-    @SuppressWarnings("unchecked")
-    I input = (I) currentTransform.getInput();
-    return input;
-  }
-
-  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
-    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-        "can only be called with current transform");
-    @SuppressWarnings("unchecked")
-    O output = (O) currentTransform.getOutput();
-    return output;
-  }
-
-  protected  <T> void setOutputRDD(PTransform<?, ?> transform,
-      JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    setRDD((PValue) getOutput(transform), rdd);
-  }
-
-  protected  <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
-      Coder<T> coder) {
-    pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder));
-  }
-
-  void setPView(PValue view, Iterable<? extends WindowedValue<?>> value) {
-    pview.put(view, value);
-  }
-
-  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
-    PValue pvalue = (PValue) getOutput(transform);
-    return pcollections.containsKey(pvalue);
-  }
-
-  protected JavaRDDLike<?, ?> getRDD(PValue pvalue) {
-    RDDHolder<?> rddHolder = pcollections.get(pvalue);
-    JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
-    leafRdds.remove(rddHolder);
-    if (multireads.contains(pvalue)) {
-      // Ensure the RDD is marked as cached
-      rdd.rdd().cache();
-    } else {
-      multireads.add(pvalue);
-    }
-    return rdd;
-  }
-
-  protected <T> void setRDD(PValue pvalue, JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    try {
-      rdd.rdd().setName(pvalue.getName());
-    } catch (IllegalStateException e) {
-      // name not set, ignore
-    }
-    RDDHolder<T> rddHolder = new RDDHolder<>(rdd);
-    pcollections.put(pvalue, rddHolder);
-    leafRdds.add(rddHolder);
-  }
-
-  JavaRDDLike<?, ?> getInputRDD(PTransform<? extends PInput, ?> transform) {
-    return getRDD((PValue) getInput(transform));
-  }
-
-
-  <T> Iterable<? extends WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
-    return pview.get(view);
-  }
-
-  /**
-   * Computes the outputs for all RDDs that are leaves in the DAG and do not have any
-   * actions (like saving to a file) registered on them (i.e. they are performed for side
-   * effects).
-   */
-  protected void computeOutputs() {
-    for (RDDHolder<?> rddHolder : leafRdds) {
-      JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
-      rdd.rdd().cache(); // cache so that any subsequent get() is cheap
-      rdd.count(); // force the RDD to be computed
-    }
-  }
-
-  @Override
-  public <T> T get(PValue value) {
-    if (pobjects.containsKey(value)) {
-      @SuppressWarnings("unchecked")
-      T result = (T) pobjects.get(value);
-      return result;
-    }
-    if (pcollections.containsKey(value)) {
-      JavaRDDLike<?, ?> rdd = pcollections.get(value).getRDD();
-      @SuppressWarnings("unchecked")
-      T res = (T) Iterables.getOnlyElement(rdd.collect());
-      pobjects.put(value, res);
-      return res;
-    }
-    throw new IllegalStateException("Cannot resolve un-known PObject: " + value);
-  }
-
-  @Override
-  public <T> T getAggregatorValue(String named, Class<T> resultType) {
-    return runtime.getAggregatorValue(named, resultType);
-  }
-
-  @Override
-  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-      throws AggregatorRetrievalException {
-    return runtime.getAggregatorValues(aggregator);
-  }
-
-  @Override
-  public <T> Iterable<T> get(PCollection<T> pcollection) {
-    @SuppressWarnings("unchecked")
-    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
-    return rddHolder.getValues(pcollection);
-  }
-
-  <T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
-    @SuppressWarnings("unchecked")
-    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
-    return rddHolder.getWindowedValues(pcollection);
-  }
-
-  @Override
-  public void close() {
-    SparkContextFactory.stopSparkContext(jsc);
-  }
-
-  /** The runner is blocking. */
-  @Override
-  public State getState() {
-    return State.DONE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java
deleted file mode 100644
index aad029a..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-/**
- * Interface for retrieving the result(s) of running a pipeline. Allows us to translate between
- * {@code PObject<T>}s or {@code PCollection<T>}s and Ts or collections of Ts.
- */
-public interface EvaluationResult extends PipelineResult {
-  /**
-   * Retrieves an iterable of results associated with the PCollection passed in.
-   *
-   * @param pcollection Collection we wish to translate.
-   * @param <T>         Type of elements contained in collection.
-   * @return Natively types result associated with collection.
-   */
-  <T> Iterable<T> get(PCollection<T> pcollection);
-
-  /**
-   * Retrieve an object of Type T associated with the PValue passed in.
-   *
-   * @param pval PValue to retrieve associated data for.
-   * @param <T>  Type of object to return.
-   * @return Native object.
-   */
-  <T> T get(PValue pval);
-
-  /**
-   * Retrieves the final value of the aggregator.
-   *
-   * @param aggName    name of aggregator.
-   * @param resultType Class of final result of aggregation.
-   * @param <T>        Type of final result of aggregation.
-   * @return Result of aggregation associated with specified name.
-   */
-  <T> T getAggregatorValue(String aggName, Class<T> resultType);
-
-  /**
-   * Releases any runtime resources, including distributed-execution contexts currently held by
-   * this EvaluationResult; once close() has been called,
-   * {@link EvaluationResult#get(PCollection)} might
-   * not work for subsequent calls.
-   */
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java
deleted file mode 100644
index d269788..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Function;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.LinkedListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-import org.joda.time.Instant;
-import scala.Tuple2;
-
-/**
- * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enriching the
- * underlying data with multiple TupleTags.
- *
- * @param <I> Input type for DoFunction.
- * @param <O> Output type for DoFunction.
- */
-class MultiDoFnFunction<I, O>
-    implements PairFlatMapFunction<Iterator<WindowedValue<I>>, TupleTag<?>, WindowedValue<?>> {
-  private final DoFn<I, O> mFunction;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final TupleTag<O> mMainOutputTag;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  MultiDoFnFunction(
-      DoFn<I, O> fn,
-      SparkRuntimeContext runtimeContext,
-      TupleTag<O> mainOutputTag,
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    this.mFunction = fn;
-    this.mRuntimeContext = runtimeContext;
-    this.mMainOutputTag = mainOutputTag;
-    this.mSideInputs = sideInputs;
-  }
-
-  @Override
-  public Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>>
-      call(Iterator<WindowedValue<I>> iter) throws Exception {
-    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
-    mFunction.startBundle(ctxt);
-    ctxt.setup();
-    return ctxt.getOutputIterable(iter, mFunction);
-  }
-
-  private class ProcCtxt extends SparkProcessContext<I, O, Tuple2<TupleTag<?>, WindowedValue<?>>> {
-
-    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
-
-    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
-        BroadcastHelper<?>> sideInputs) {
-      super(fn, runtimeContext, sideInputs);
-    }
-
-    @Override
-    public synchronized void output(O o) {
-      outputs.put(mMainOutputTag, windowedValue.withValue(o));
-    }
-
-    @Override
-    public synchronized void output(WindowedValue<O> o) {
-      outputs.put(mMainOutputTag, o);
-    }
-
-    @Override
-    public synchronized <T> void sideOutput(TupleTag<T> tag, T t) {
-      outputs.put(tag, windowedValue.withValue(t));
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
-      outputs.put(tupleTag, WindowedValue.of(t, instant,
-          windowedValue.getWindows(), windowedValue.getPane()));
-    }
-
-    @Override
-    protected void clearOutput() {
-      outputs.clear();
-    }
-
-    @Override
-    protected Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> getOutputIterator() {
-      return Iterators.transform(outputs.entries().iterator(),
-          new Function<Map.Entry<TupleTag<?>, WindowedValue<?>>,
-              Tuple2<TupleTag<?>, WindowedValue<?>>>() {
-        @Override
-        public Tuple2<TupleTag<?>, WindowedValue<?>> apply(Map.Entry<TupleTag<?>,
-            WindowedValue<?>> input) {
-          return new Tuple2<TupleTag<?>, WindowedValue<?>>(input.getKey(), input.getValue());
-        }
-      });
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java
deleted file mode 100644
index f53b6d9..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.fs.Path;
-
-final class ShardNameBuilder {
-
-  private ShardNameBuilder() {
-  }
-
-  /**
-   * Replace occurrences of uppercase letters 'N' with the given {code}shardCount{code},
-   * left-padded with zeros if necessary.
-   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
-   * @param template the string template containing uppercase letters 'N'
-   * @param shardCount the total number of shards
-   * @return a string template with 'N' replaced by the shard count
-   */
-  public static String replaceShardCount(String template, int shardCount) {
-    return replaceShardPattern(template, "N+", shardCount);
-  }
-
-  /**
-   * Replace occurrences of uppercase letters 'S' with the given {code}shardNumber{code},
-   * left-padded with zeros if necessary.
-   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
-   * @param template the string template containing uppercase letters 'S'
-   * @param shardNumber the number of a particular shard
-   * @return a string template with 'S' replaced by the shard number
-   */
-  public static String replaceShardNumber(String template, int shardNumber) {
-    return replaceShardPattern(template, "S+", shardNumber);
-  }
-
-  private static String replaceShardPattern(String template, String pattern, int n) {
-    Pattern p = Pattern.compile(pattern);
-    Matcher m = p.matcher(template);
-    StringBuffer sb = new StringBuffer();
-    while (m.find()) {
-      // replace pattern with a String format string:
-      // index 1, zero-padding flag (0), width length of matched pattern, decimal conversion
-      m.appendReplacement(sb, "%1\\$0" + m.group().length() + "d");
-    }
-    m.appendTail(sb);
-    return String.format(sb.toString(), n);
-  }
-
-  /**
-   * @param pathPrefix a relative or absolute path
-   * @param template a template string
-   * @return the output directory for the given prefix, template and suffix
-   */
-  public static String getOutputDirectory(String pathPrefix, String template) {
-    String out = new Path(pathPrefix + template).getParent().toString();
-    if (out.isEmpty()) {
-      return "./";
-    }
-    return out;
-  }
-
-  /**
-   * @param pathPrefix a relative or absolute path
-   * @param template a template string
-   * @return the prefix of the output filename for the given path prefix and template
-   */
-  public static String getOutputFilePrefix(String pathPrefix, String template) {
-    String name = new Path(pathPrefix + template).getName();
-    if (name.endsWith(template)) {
-      return name.substring(0, name.length() - template.length());
-    } else {
-      return "";
-    }
-  }
-
-  /**
-   * @param pathPrefix a relative or absolute path
-   * @param template a template string
-   * @return the template for the output filename for the given path prefix and
-   * template
-   */
-  public static String getOutputFileTemplate(String pathPrefix, String template) {
-    String name = new Path(pathPrefix + template).getName();
-    if (name.endsWith(template)) {
-      return template;
-    } else {
-      return name;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateAware.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateAware.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateAware.java
deleted file mode 100644
index bb9a7a5..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateAware.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-/**
- * A marker interface that implementations of
- * {@link org.apache.hadoop.mapreduce.lib.output.FileOutputFormat} implement to indicate
- * that they produce shard names that adhere to the template in
- * {@link com.cloudera.dataflow.hadoop.HadoopIO.Write}.
- *
- * Some common shard names are defined in
- * {@link com.google.cloud.dataflow.sdk.io.ShardNameTemplate}.
- */
-public interface ShardNameTemplateAware {
-}


[16/23] incubator-beam git commit: [BEAM-11] second iteration of package reorganisation

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
new file mode 100644
index 0000000..6ba04b7
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation;
+
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.spark.api.java.function.Function;
+
+/**
+ * Helper functions for working with windows.
+ */
+public final class WindowingHelpers {
+  private WindowingHelpers() {
+  }
+
+  /**
+   * A function for converting a value to a {@link WindowedValue}. The resulting
+   * {@link WindowedValue} will be in no windows, and will have the default timestamp
+   * and pane.
+   *
+   * @param <T>   The type of the object.
+   * @return A function that accepts an object and returns its {@link WindowedValue}.
+   */
+  public static <T> Function<T, WindowedValue<T>> windowFunction() {
+    return new Function<T, WindowedValue<T>>() {
+      @Override
+      public WindowedValue<T> call(T t) {
+        return WindowedValue.valueInEmptyWindows(t);
+      }
+    };
+  }
+
+  /**
+   * A function for extracting the value from a {@link WindowedValue}.
+   *
+   * @param <T>   The type of the object.
+   * @return A function that accepts a {@link WindowedValue} and returns its value.
+   */
+  public static <T> Function<WindowedValue<T>, T> unwindowFunction() {
+    return new Function<WindowedValue<T>, T>() {
+      @Override
+      public T call(WindowedValue<T> t) {
+        return t.getValue();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
new file mode 100644
index 0000000..2c34caf
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+
+public final class SparkStreamingPipelineOptionsFactory {
+
+  private SparkStreamingPipelineOptionsFactory() {
+  }
+
+  public static SparkStreamingPipelineOptions create() {
+    return PipelineOptionsFactory.as(SparkStreamingPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
new file mode 100644
index 0000000..e39d3ed
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+
+public class SparkStreamingPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
+
+  @Override
+  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkStreamingPipelineOptions
+            .class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/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
new file mode 100644
index 0000000..0e87355
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+
+import org.apache.beam.runners.spark.translation.EvaluationContext;
+import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaDStreamLike;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+
+/**
+ * Streaming evaluation context helps to handle streaming.
+ */
+public class StreamingEvaluationContext extends EvaluationContext {
+
+  private final JavaStreamingContext jssc;
+  private final long timeout;
+  private final Map<PValue, DStreamHolder<?>> pstreams = new LinkedHashMap<>();
+  private final Set<DStreamHolder<?>> leafStreams = new LinkedHashSet<>();
+
+  public StreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
+      JavaStreamingContext jssc, long timeout) {
+    super(jsc, pipeline);
+    this.jssc = jssc;
+    this.timeout = timeout;
+  }
+
+  /**
+   * DStream holder Can also crate a DStream from a supplied queue of values, but mainly for
+   * testing.
+   */
+  private class DStreamHolder<T> {
+
+    private Iterable<Iterable<T>> values;
+    private Coder<T> coder;
+    private JavaDStream<WindowedValue<T>> dStream;
+
+    DStreamHolder(Iterable<Iterable<T>> values, Coder<T> coder) {
+      this.values = values;
+      this.coder = coder;
+    }
+
+    DStreamHolder(JavaDStream<WindowedValue<T>> dStream) {
+      this.dStream = dStream;
+    }
+
+    @SuppressWarnings("unchecked")
+    JavaDStream<WindowedValue<T>> getDStream() {
+      if (dStream == null) {
+        // create the DStream from values
+        Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
+        for (Iterable<T> v : values) {
+          setOutputRDDFromValues(currentTransform.getTransform(), v, coder);
+          rddQueue.offer((JavaRDD<WindowedValue<T>>) getOutputRDD(currentTransform.getTransform()));
+        }
+        // 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);
+      }
+      return dStream;
+    }
+  }
+
+  <T> void setDStreamFromQueue(
+      PTransform<?, ?> transform, Iterable<Iterable<T>> values, Coder<T> coder) {
+    pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder));
+  }
+
+  <T> void setStream(PTransform<?, ?> transform, JavaDStream<WindowedValue<T>> dStream) {
+    PValue pvalue = (PValue) getOutput(transform);
+    DStreamHolder<T> dStreamHolder = new DStreamHolder<>(dStream);
+    pstreams.put(pvalue, dStreamHolder);
+    leafStreams.add(dStreamHolder);
+  }
+
+  boolean hasStream(PTransform<?, ?> transform) {
+    PValue pvalue = (PValue) getInput(transform);
+    return pstreams.containsKey(pvalue);
+  }
+
+  JavaDStreamLike<?, ?, ?> getStream(PTransform<?, ?> transform) {
+    return getStream((PValue) getInput(transform));
+  }
+
+  JavaDStreamLike<?, ?, ?> getStream(PValue pvalue) {
+    DStreamHolder<?> dStreamHolder = pstreams.get(pvalue);
+    JavaDStreamLike<?, ?, ?> dStream = dStreamHolder.getDStream();
+    leafStreams.remove(dStreamHolder);
+    return dStream;
+  }
+
+  // used to set the RDD from the DStream in the RDDHolder for transformation
+  <T> void setInputRDD(
+      PTransform<? extends PInput, ?> transform, JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    setRDD((PValue) getInput(transform), rdd);
+  }
+
+  // used to get the RDD transformation output and use it as the DStream transformation output
+  JavaRDDLike<?, ?> getOutputRDD(PTransform<?, ?> transform) {
+    return getRDD((PValue) getOutput(transform));
+  }
+
+  public JavaStreamingContext getStreamingContext() {
+    return jssc;
+  }
+
+  @Override
+  public void computeOutputs() {
+    for (DStreamHolder<?> streamHolder : leafStreams) {
+      computeOutput(streamHolder);
+    }
+  }
+
+  private static <T> void computeOutput(DStreamHolder<T> streamHolder) {
+    streamHolder.getDStream().foreachRDD(new Function<JavaRDD<WindowedValue<T>>, Void>() {
+      @Override
+      public Void call(JavaRDD<WindowedValue<T>> rdd) throws Exception {
+        rdd.rdd().cache();
+        rdd.count();
+        return null;
+      }
+    }); // force a DStream action
+  }
+
+  @Override
+  public void close() {
+    if (timeout > 0) {
+      jssc.awaitTerminationOrTimeout(timeout);
+    } else {
+      jssc.awaitTermination();
+    }
+    //TODO: stop gracefully ?
+    jssc.stop(false, false);
+    state = State.DONE;
+    super.close();
+  }
+
+  private State state = State.RUNNING;
+
+  @Override
+  public State getState() {
+    return state;
+  }
+
+  //---------------- override in order to expose in package
+  @Override
+  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
+    return super.getInput(transform);
+  }
+  @Override
+  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
+    return super.getOutput(transform);
+  }
+
+  @Override
+  protected JavaSparkContext getSparkContext() {
+    return super.getSparkContext();
+  }
+
+  @Override
+  protected SparkRuntimeContext getRuntimeContext() {
+    return super.getRuntimeContext();
+  }
+
+  @Override
+  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
+    super.setCurrentTransform(transform);
+  }
+
+  @Override
+  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
+    return super.getCurrentTransform();
+  }
+
+  @Override
+  protected <T> void setOutputRDD(PTransform<?, ?> transform,
+      JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    super.setOutputRDD(transform, rdd);
+  }
+
+  @Override
+  protected <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
+      Coder<T> coder) {
+    super.setOutputRDDFromValues(transform, values, coder);
+  }
+
+  @Override
+  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
+    return super.hasOutputRDD(transform);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
new file mode 100644
index 0000000..349bb7c
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming;
+
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.api.client.util.Lists;
+import com.google.api.client.util.Maps;
+import com.google.api.client.util.Sets;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+import com.google.common.reflect.TypeToken;
+import kafka.serializer.Decoder;
+
+import org.apache.beam.runners.spark.io.ConsoleIO;
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.io.KafkaIO;
+import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
+import org.apache.beam.runners.spark.translation.DoFnFunction;
+import org.apache.beam.runners.spark.translation.EvaluationContext;
+import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
+import org.apache.beam.runners.spark.translation.TransformEvaluator;
+import org.apache.beam.runners.spark.translation.TransformTranslator;
+import org.apache.beam.runners.spark.translation.WindowingHelpers;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaDStreamLike;
+import org.apache.spark.streaming.api.java.JavaPairInputDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.kafka.KafkaUtils;
+
+import scala.Tuple2;
+
+
+/**
+ * Supports translation between a DataFlow transform, and Spark's operations on DStreams.
+ */
+public final class StreamingTransformTranslator {
+
+  private StreamingTransformTranslator() {
+  }
+
+  private static <T> TransformEvaluator<ConsoleIO.Write.Unbound<T>> print() {
+    return new TransformEvaluator<ConsoleIO.Write.Unbound<T>>() {
+      @Override
+      public void evaluate(ConsoleIO.Write.Unbound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
+            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
+            ((StreamingEvaluationContext) context).getStream(transform);
+        dstream.map(WindowingHelpers.<T>unwindowFunction()).print(transform.getNum());
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<KafkaIO.Read.Unbound<K, V>> kafka() {
+    return new TransformEvaluator<KafkaIO.Read.Unbound<K, V>>() {
+      @Override
+      public void evaluate(KafkaIO.Read.Unbound<K, V> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        JavaStreamingContext jssc = sec.getStreamingContext();
+        Class<K> keyClazz = transform.getKeyClass();
+        Class<V> valueClazz = transform.getValueClass();
+        Class<? extends Decoder<K>> keyDecoderClazz = transform.getKeyDecoderClass();
+        Class<? extends Decoder<V>> valueDecoderClazz = transform.getValueDecoderClass();
+        Map<String, String> kafkaParams = transform.getKafkaParams();
+        Set<String> topics = transform.getTopics();
+        JavaPairInputDStream<K, V> inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz,
+                valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics);
+        JavaDStream<WindowedValue<KV<K, V>>> inputStream =
+            inputPairStream.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+          @Override
+          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
+            return KV.of(t2._1(), t2._2());
+          }
+        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
+        sec.setStream(transform, inputStream);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Create.Values<T>> create() {
+    return new TransformEvaluator<Create.Values<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        Iterable<T> elems = transform.getElements();
+        Coder<T> coder = sec.getOutput(transform).getCoder();
+        if (coder != VoidCoder.of()) {
+          // actual create
+          sec.setOutputRDDFromValues(transform, elems, coder);
+        } else {
+          // fake create as an input
+          // creates a stream with a single batch containing a single null element
+          // to invoke following transformations once
+          // to support DataflowAssert
+          sec.setDStreamFromQueue(transform,
+              Collections.<Iterable<Void>>singletonList(Collections.singletonList((Void) null)),
+              (Coder<Void>) coder);
+        }
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<CreateStream.QueuedValues<T>> createFromQueue() {
+    return new TransformEvaluator<CreateStream.QueuedValues<T>>() {
+      @Override
+      public void evaluate(CreateStream.QueuedValues<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        Iterable<Iterable<T>> values = transform.getQueuedValues();
+        Coder<T> coder = sec.getOutput(transform).getCoder();
+        sec.setDStreamFromQueue(transform, values, coder);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
+    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        PCollectionList<T> pcs = sec.getInput(transform);
+        JavaDStream<WindowedValue<T>> first =
+            (JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(0));
+        List<JavaDStream<WindowedValue<T>>> rest = Lists.newArrayListWithCapacity(pcs.size() - 1);
+        for (int i = 1; i < pcs.size(); i++) {
+          rest.add((JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(i)));
+        }
+        JavaDStream<WindowedValue<T>> dstream = sec.getStreamingContext().union(first, rest);
+        sec.setStream(transform, dstream);
+      }
+    };
+  }
+
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> rddTransform(
+      final SparkPipelineTranslator rddTranslator) {
+    return new TransformEvaluator<PT>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator<PT> rddEvaluator =
+            rddTranslator.translate((Class<PT>) transform.getClass());
+
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        if (sec.hasStream(transform)) {
+          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
+              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
+              sec.getStream(transform);
+
+          sec.setStream(transform, dStream
+              .transform(new RDDTransform<>(sec, rddEvaluator, transform)));
+        } else {
+          // if the transformation requires direct access to RDD (not in stream)
+          // this is used for "fake" transformations like with DataflowAssert
+          rddEvaluator.evaluate(transform, context);
+        }
+      }
+    };
+  }
+
+  /**
+   * RDD transform function If the transformation function doesn't have an input, create a fake one
+   * as an empty RDD.
+   *
+   * @param <PT> PTransform type
+   */
+  private static final class RDDTransform<PT extends PTransform<?, ?>>
+      implements Function<JavaRDD<WindowedValue<Object>>, JavaRDD<WindowedValue<Object>>> {
+
+    private final StreamingEvaluationContext context;
+    private final AppliedPTransform<?, ?, ?> appliedPTransform;
+    private final TransformEvaluator<PT> rddEvaluator;
+    private final PT transform;
+
+
+    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator<PT> rddEvaluator,
+        PT transform) {
+      this.context = context;
+      this.appliedPTransform = context.getCurrentTransform();
+      this.rddEvaluator = rddEvaluator;
+      this.transform = transform;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public JavaRDD<WindowedValue<Object>>
+        call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
+      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
+      context.setCurrentTransform(appliedPTransform);
+      context.setInputRDD(transform, rdd);
+      rddEvaluator.evaluate(transform, context);
+      if (!context.hasOutputRDD(transform)) {
+        // fake RDD as output
+        context.setOutputRDD(transform,
+            context.getSparkContext().<WindowedValue<Object>>emptyRDD());
+      }
+      JavaRDD<WindowedValue<Object>> outRDD =
+          (JavaRDD<WindowedValue<Object>>) context.getOutputRDD(transform);
+      context.setCurrentTransform(existingAPT);
+      return outRDD;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> foreachRDD(
+      final SparkPipelineTranslator rddTranslator) {
+    return new TransformEvaluator<PT>() {
+      @Override
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator<PT> rddEvaluator =
+            rddTranslator.translate((Class<PT>) transform.getClass());
+
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        if (sec.hasStream(transform)) {
+          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
+              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
+              sec.getStream(transform);
+
+          dStream.foreachRDD(new RDDOutputOperator<>(sec, rddEvaluator, transform));
+        } else {
+          rddEvaluator.evaluate(transform, context);
+        }
+      }
+    };
+  }
+
+  /**
+   * RDD output function.
+   *
+   * @param <PT> PTransform type
+   */
+  private static final class RDDOutputOperator<PT extends PTransform<?, ?>>
+      implements Function<JavaRDD<WindowedValue<Object>>, Void> {
+
+    private final StreamingEvaluationContext context;
+    private final AppliedPTransform<?, ?, ?> appliedPTransform;
+    private final TransformEvaluator<PT> rddEvaluator;
+    private final PT transform;
+
+
+    private RDDOutputOperator(StreamingEvaluationContext context,
+        TransformEvaluator<PT> rddEvaluator, PT transform) {
+      this.context = context;
+      this.appliedPTransform = context.getCurrentTransform();
+      this.rddEvaluator = rddEvaluator;
+      this.transform = transform;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Void call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
+      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
+      context.setCurrentTransform(appliedPTransform);
+      context.setInputRDD(transform, rdd);
+      rddEvaluator.evaluate(transform, context);
+      context.setCurrentTransform(existingAPT);
+      return null;
+    }
+  }
+
+  private static final TransformTranslator.FieldGetter WINDOW_FG =
+      new TransformTranslator.FieldGetter(Window.Bound.class);
+
+  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
+    return new TransformEvaluator<Window.Bound<T>>() {
+      @Override
+      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        //--- first we apply windowing to the stream
+        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
+        @SuppressWarnings("unchecked")
+        JavaDStream<WindowedValue<T>> dStream =
+            (JavaDStream<WindowedValue<T>>) sec.getStream(transform);
+        if (windowFn instanceof FixedWindows) {
+          Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize()
+              .getMillis());
+          sec.setStream(transform, dStream.window(windowDuration));
+        } else if (windowFn instanceof SlidingWindows) {
+          Duration windowDuration = Durations.milliseconds(((SlidingWindows) windowFn).getSize()
+              .getMillis());
+          Duration slideDuration = Durations.milliseconds(((SlidingWindows) windowFn).getPeriod()
+              .getMillis());
+          sec.setStream(transform, dStream.window(windowDuration, slideDuration));
+        }
+        //--- then we apply windowing to the elements
+        DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
+        DoFnFunction<T, T> dofn = new DoFnFunction<>(addWindowsDoFn,
+            ((StreamingEvaluationContext)context).getRuntimeContext(), null);
+        @SuppressWarnings("unchecked")
+        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
+            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
+            sec.getStream(transform);
+        sec.setStream(transform, dstream.mapPartitions(dofn));
+      }
+    };
+  }
+
+  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
+      .newHashMap();
+
+  static {
+    EVALUATORS.put(ConsoleIO.Write.Unbound.class, print());
+    EVALUATORS.put(CreateStream.QueuedValues.class, createFromQueue());
+    EVALUATORS.put(Create.Values.class, create());
+    EVALUATORS.put(KafkaIO.Read.Unbound.class, kafka());
+    EVALUATORS.put(Window.Bound.class, window());
+    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
+  }
+
+  private static final Set<Class<? extends PTransform>> UNSUPPORTED_EVALUATORS = Sets
+      .newHashSet();
+
+  static {
+    //TODO - add support for the following
+    UNSUPPORTED_EVALUATORS.add(TextIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(TextIO.Write.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(AvroIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(AvroIO.Write.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(HadoopIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(HadoopIO.Write.Bound.class);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
+      getTransformEvaluator(Class<PT> clazz, SparkPipelineTranslator rddTranslator) {
+    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
+    if (transform == null) {
+      if (UNSUPPORTED_EVALUATORS.contains(clazz)) {
+        throw new UnsupportedOperationException("Dataflow transformation " + clazz
+          .getCanonicalName()
+          + " is currently unsupported by the Spark streaming pipeline");
+      }
+      // DStream transformations will transform an RDD into another RDD
+      // Actions will create output
+      // In Dataflow it depends on the PTransform's Input and Output class
+      Class<?> pTOutputClazz = getPTransformOutputClazz(clazz);
+      if (PDone.class.equals(pTOutputClazz)) {
+        return foreachRDD(rddTranslator);
+      } else {
+        return rddTransform(rddTranslator);
+      }
+    }
+    return transform;
+  }
+
+  private static <PT extends PTransform<?, ?>> Class<?> getPTransformOutputClazz(Class<PT> clazz) {
+    Type[] types = ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments();
+    return TypeToken.of(clazz).resolveType(types[1]).getRawType();
+  }
+
+  /**
+   * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator.
+   * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation
+   */
+  public static class Translator implements SparkPipelineTranslator {
+
+    private final SparkPipelineTranslator rddTranslator;
+
+    public Translator(SparkPipelineTranslator rddTranslator) {
+      this.rddTranslator = rddTranslator;
+    }
+
+    @Override
+    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
+      // streaming includes rdd transformations as well
+      return EVALUATORS.containsKey(clazz) || rddTranslator.hasTranslation(clazz);
+    }
+
+    @Override
+    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
+      return getTransformEvaluator(clazz, rddTranslator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
new file mode 100644
index 0000000..504ea92
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.translation.streaming;
+
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
+import org.apache.beam.runners.spark.translation.TransformTranslator;
+
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Durations;
+
+
+/**
+ * Pipeline {@link SparkPipelineRunner.Evaluator} to detect windowing.
+ */
+public final class StreamingWindowPipelineDetector extends SparkPipelineRunner.Evaluator {
+
+  // Currently, Spark streaming recommends batches no smaller then 500 msec
+  private static final Duration SPARK_MIN_WINDOW = Durations.milliseconds(500);
+
+  private boolean windowing;
+  private Duration batchDuration;
+
+  public StreamingWindowPipelineDetector(SparkPipelineTranslator translator) {
+    super(translator);
+  }
+
+  private static final TransformTranslator.FieldGetter WINDOW_FG =
+      new TransformTranslator.FieldGetter(Window.Bound.class);
+
+  // Use the smallest window (fixed or sliding) as Spark streaming's batch duration
+  @Override
+  protected <PT extends PTransform<? super PInput, POutput>> void
+      doVisitTransform(TransformTreeNode node) {
+    @SuppressWarnings("unchecked")
+    PT transform = (PT) node.getTransform();
+    @SuppressWarnings("unchecked")
+    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
+    if (transformClass.isAssignableFrom(Window.Bound.class)) {
+      WindowFn<?, ?> windowFn = WINDOW_FG.get("windowFn", transform);
+      if (windowFn instanceof FixedWindows) {
+        setBatchDuration(((FixedWindows) windowFn).getSize());
+      } else if (windowFn instanceof SlidingWindows) {
+        if (((SlidingWindows) windowFn).getOffset().getMillis() > 0) {
+          throw new UnsupportedOperationException("Spark does not support window offsets");
+        }
+        // Sliding window size might as well set the batch duration. Applying the transformation
+        // will add the "slide"
+        setBatchDuration(((SlidingWindows) windowFn).getSize());
+      } else if (!(windowFn instanceof GlobalWindows)) {
+        throw new IllegalStateException("Windowing function not supported: " + windowFn);
+      }
+    }
+  }
+
+  private void setBatchDuration(org.joda.time.Duration duration) {
+    Long durationMillis = duration.getMillis();
+    // validate window size
+    if (durationMillis < SPARK_MIN_WINDOW.milliseconds()) {
+      throw new IllegalArgumentException("Windowing of size " + durationMillis +
+          "msec is not supported!");
+    }
+    // choose the smallest duration to be Spark's batch duration, larger ones will be handled
+    // as window functions  over the batched-stream
+    if (!windowing || this.batchDuration.milliseconds() > durationMillis) {
+      this.batchDuration = Durations.milliseconds(durationMillis);
+    }
+    windowing = true;
+  }
+
+  public boolean isWindowing() {
+    return windowing;
+  }
+
+  public Duration getBatchDuration() {
+    return batchDuration;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
index 98387a6..e4a3a73 100644
--- a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
+++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
@@ -13,5 +13,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-org.apache.beam.runners.spark.SparkPipelineOptionsRegistrar
-org.apache.beam.runners.spark.streaming.SparkStreamingPipelineOptionsRegistrar
\ No newline at end of file
+org.apache.beam.runners.spark.translation.SparkPipelineOptionsRegistrar
+org.apache.beam.runners.spark.translation.streaming.SparkStreamingPipelineOptionsRegistrar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
index 972b1a3..7949db4 100644
--- a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
+++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-org.apache.beam.runners.spark.SparkPipelineRunnerRegistrar
\ No newline at end of file
+org.apache.beam.runners.spark.translation.SparkPipelineRunnerRegistrar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
deleted file mode 100644
index 35a634a..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Iterables;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class CombineGloballyTest {
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    Pipeline p = Pipeline.create(options);
-    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-    PCollection<String> output = inputWords.apply(Combine.globally(new WordMerger()));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    assertEquals("hi there,hi,hi sue bob,hi sue,,bob hi", Iterables.getOnlyElement(res.get(output)));
-    res.close();
-  }
-
-  public static class WordMerger extends Combine.CombineFn<String, StringBuilder, String> {
-
-    @Override
-    public StringBuilder createAccumulator() {
-      // return null to differentiate from an empty string
-      return null;
-    }
-
-    @Override
-    public StringBuilder addInput(StringBuilder accumulator, String input) {
-      return combine(accumulator, input);
-    }
-
-    @Override
-    public StringBuilder mergeAccumulators(Iterable<StringBuilder> accumulators) {
-      StringBuilder sb = new StringBuilder();
-      for (StringBuilder accum : accumulators) {
-        if (accum != null) {
-          sb.append(accum);
-        }
-      }
-      return sb;
-    }
-
-    @Override
-    public String extractOutput(StringBuilder accumulator) {
-      return accumulator != null ? accumulator.toString(): "";
-    }
-
-    private static StringBuilder combine(StringBuilder accum, String datum) {
-      if (accum == null) {
-        return new StringBuilder(datum);
-      } else {
-        accum.append(",").append(datum);
-        return accum;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
deleted file mode 100644
index a4c5eb7..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class CombinePerKeyTest {
-
-    private static final List<String> WORDS =
-        ImmutableList.of("the", "quick", "brown", "fox", "jumped", "over", "the", "lazy", "dog");
-    @Test
-    public void testRun() {
-        Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-        PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-        PCollection<KV<String, Long>> cnts = inputWords.apply(new SumPerKey<String>());
-        EvaluationResult res = SparkPipelineRunner.create().run(p);
-        Map<String, Long> actualCnts = new HashMap<>();
-        for (KV<String, Long> kv : res.get(cnts)) {
-            actualCnts.put(kv.getKey(), kv.getValue());
-        }
-        res.close();
-        Assert.assertEquals(8, actualCnts.size());
-        Assert.assertEquals(Long.valueOf(2L), actualCnts.get("the"));
-    }
-
-    private static class SumPerKey<T> extends PTransform<PCollection<T>, PCollection<KV<T, Long>>> {
-      @Override
-      public PCollection<KV<T, Long>> apply(PCollection<T> pcol) {
-          PCollection<KV<T, Long>> withLongs = pcol.apply(ParDo.of(new DoFn<T, KV<T, Long>>() {
-              @Override
-              public void processElement(ProcessContext processContext) throws Exception {
-                  processContext.output(KV.of(processContext.element(), 1L));
-              }
-          })).setCoder(KvCoder.of(pcol.getCoder(), VarLongCoder.of()));
-          return withLongs.apply(Sum.<T>longsPerKey());
-      }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
index 905e243..4a080e8 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
@@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.Create;
 import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
 import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
 import org.junit.Test;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
deleted file mode 100644
index 1ec3d75..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import org.junit.Test;
-
-import java.io.Serializable;
-
-public class DoFnOutputTest implements Serializable {
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    PCollection<String> strings = pipeline.apply(Create.of("a"));
-    // Test that values written from startBundle() and finishBundle() are written to
-    // the output
-    PCollection<String> output = strings.apply(ParDo.of(new DoFn<String, String>() {
-      @Override
-      public void startBundle(Context c) throws Exception {
-        c.output("start");
-      }
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(c.element());
-      }
-      @Override
-      public void finishBundle(Context c) throws Exception {
-        c.output("finish");
-      }
-    }));
-
-    DataflowAssert.that(output).containsInAnyOrder("start", "a", "finish");
-
-    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
-    res.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
index e0fe47d..057cf3b 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
@@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.Create;
 import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
 import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
 import org.junit.Test;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
deleted file mode 100644
index c89090d..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.*;
-import com.google.common.collect.Iterables;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class MultiOutputWordCountTest {
-
-  private static final TupleTag<String> upper = new TupleTag<>();
-  private static final TupleTag<String> lower = new TupleTag<>();
-  private static final TupleTag<KV<String, Long>> lowerCnts = new TupleTag<>();
-  private static final TupleTag<KV<String, Long>> upperCnts = new TupleTag<>();
-
-  @Test
-  public void testRun() throws Exception {
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    PCollection<String> regex = p.apply(Create.of("[^a-zA-Z']+"));
-    PCollection<String> w1 = p.apply(Create.of("Here are some words to count", "and some others"));
-    PCollection<String> w2 = p.apply(Create.of("Here are some more words", "and even more words"));
-    PCollectionList<String> list = PCollectionList.of(w1).and(w2);
-
-    PCollection<String> union = list.apply(Flatten.<String>pCollections());
-    PCollectionView<String> regexView = regex.apply(View.<String>asSingleton());
-    CountWords countWords = new CountWords(regexView);
-    PCollectionTuple luc = union.apply(countWords);
-    PCollection<Long> unique = luc.get(lowerCnts).apply(
-        ApproximateUnique.<KV<String, Long>>globally(16));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    Iterable<KV<String, Long>> actualLower = res.get(luc.get(lowerCnts));
-    Assert.assertEquals("are", actualLower.iterator().next().getKey());
-    Iterable<KV<String, Long>> actualUpper = res.get(luc.get(upperCnts));
-    Assert.assertEquals("Here", actualUpper.iterator().next().getKey());
-    Iterable<Long> actualUniqCount = res.get(unique);
-    Assert.assertEquals(9, (long) actualUniqCount.iterator().next());
-    int actualTotalWords = res.getAggregatorValue("totalWords", Integer.class);
-    Assert.assertEquals(18, actualTotalWords);
-    int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class);
-    Assert.assertEquals(6, actualMaxWordLength);
-    AggregatorValues<Integer> aggregatorValues = res.getAggregatorValues(countWords
-        .getTotalWordsAggregator());
-    Assert.assertEquals(18, Iterables.getOnlyElement(aggregatorValues.getValues()).intValue());
-
-    res.close();
-  }
-
-  /**
-   * A DoFn that tokenizes lines of text into individual words.
-   */
-  static class ExtractWordsFn extends DoFn<String, String> {
-
-    private final Aggregator<Integer, Integer> totalWords = createAggregator("totalWords",
-        new Sum.SumIntegerFn());
-    private final Aggregator<Integer, Integer> maxWordLength = createAggregator("maxWordLength",
-        new Max.MaxIntegerFn());
-    private final PCollectionView<String> regex;
-
-    ExtractWordsFn(PCollectionView<String> regex) {
-      this.regex = regex;
-    }
-
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] words = c.element().split(c.sideInput(regex));
-      for (String word : words) {
-        totalWords.addValue(1);
-        if (!word.isEmpty()) {
-          maxWordLength.addValue(word.length());
-          if (Character.isLowerCase(word.charAt(0))) {
-            c.output(word);
-          } else {
-            c.sideOutput(upper, word);
-          }
-        }
-      }
-    }
-  }
-
-  public static class CountWords extends PTransform<PCollection<String>, PCollectionTuple> {
-
-    private final PCollectionView<String> regex;
-    private final ExtractWordsFn extractWordsFn;
-
-    public CountWords(PCollectionView<String> regex) {
-      this.regex = regex;
-      this.extractWordsFn = new ExtractWordsFn(regex);
-    }
-
-    @Override
-    public PCollectionTuple apply(PCollection<String> lines) {
-      // Convert lines of text into individual words.
-      PCollectionTuple lowerUpper = lines
-          .apply(ParDo.of(extractWordsFn)
-              .withSideInputs(regex)
-              .withOutputTags(lower, TupleTagList.of(upper)));
-      lowerUpper.get(lower).setCoder(StringUtf8Coder.of());
-      lowerUpper.get(upper).setCoder(StringUtf8Coder.of());
-      PCollection<KV<String, Long>> lowerCounts = lowerUpper.get(lower).apply(Count
-          .<String>perElement());
-      PCollection<KV<String, Long>> upperCounts = lowerUpper.get(upper).apply(Count
-          .<String>perElement());
-      return PCollectionTuple
-          .of(lowerCnts, lowerCounts)
-          .and(upperCnts, upperCounts);
-    }
-
-    Aggregator<Integer, Integer> getTotalWordsAggregator() {
-      return extractWordsFn.totalWords;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
deleted file mode 100644
index ae1eed7..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-public class SerializationTest {
-
-  public static class StringHolder { // not serializable
-    private final String string;
-
-    public StringHolder(String string) {
-      this.string = string;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      StringHolder that = (StringHolder) o;
-      return string.equals(that.string);
-    }
-
-    @Override
-    public int hashCode() {
-      return string.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return string;
-    }
-  }
-
-  public static class StringHolderUtf8Coder extends AtomicCoder<StringHolder> {
-
-    private final StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of();
-
-    @Override
-    public void encode(StringHolder value, OutputStream outStream, Context context) throws IOException {
-      stringUtf8Coder.encode(value.toString(), outStream, context);
-    }
-
-    @Override
-    public StringHolder decode(InputStream inStream, Context context) throws IOException {
-      return new StringHolder(stringUtf8Coder.decode(inStream, context));
-    }
-
-    public static Coder<StringHolder> of() {
-      return new StringHolderUtf8Coder();
-    }
-  }
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<StringHolder> WORDS = Lists.transform(
-      Arrays.asList(WORDS_ARRAY), new Function<String, StringHolder>() {
-        @Override public StringHolder apply(String s) {
-          return new StringHolder(s);
-        }
-      });
-  private static final Set<StringHolder> EXPECTED_COUNT_SET =
-      ImmutableSet.copyOf(Lists.transform(
-          Arrays.asList("hi: 5", "there: 1", "sue: 2", "bob: 2"),
-          new Function<String, StringHolder>() {
-            @Override
-            public StringHolder apply(String s) {
-              return new StringHolder(s);
-            }
-          }));
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    PCollection<StringHolder> inputWords =
-        p.apply(Create.of(WORDS).withCoder(StringHolderUtf8Coder.of()));
-    PCollection<StringHolder> output = inputWords.apply(new CountWords());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-
-  /**
-   * A DoFn that tokenizes lines of text into individual words.
-   */
-  static class ExtractWordsFn extends DoFn<StringHolder, StringHolder> {
-    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      // Split the line into words.
-      String[] words = WORD_BOUNDARY.split(c.element().toString());
-
-      // Keep track of the number of lines without any words encountered while tokenizing.
-      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
-      if (words.length == 0) {
-        emptyLines.addValue(1L);
-      }
-
-      // Output each word encountered into the output PCollection.
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(new StringHolder(word));
-        }
-      }
-    }
-  }
-
-  /**
-   * A DoFn that converts a Word and Count into a printable string.
-   */
-  private static class FormatCountsFn extends DoFn<KV<StringHolder, Long>, StringHolder> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(new StringHolder(c.element().getKey() + ": " + c.element().getValue()));
-    }
-  }
-
-  private static class CountWords extends PTransform<PCollection<StringHolder>, PCollection<StringHolder>> {
-    @Override
-    public PCollection<StringHolder> apply(PCollection<StringHolder> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<StringHolder> words = lines.apply(
-          ParDo.of(new ExtractWordsFn()));
-
-      // Count the number of times each word occurs.
-      PCollection<KV<StringHolder, Long>> wordCounts =
-          words.apply(Count.<StringHolder>perElement());
-
-      // Format each word and count into a printable string.
-
-      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
deleted file mode 100644
index bdc048c..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.net.URI;
-
-import static org.junit.Assert.*;
-
-public class SideEffectsTest implements Serializable {
-
-  static class UserException extends RuntimeException {
-  }
-
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
-    pipeline.apply(Create.of("a")).apply(ParDo.of(new DoFn<String, String>() {
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        throw new UserException();
-      }
-    }));
-
-    try {
-      pipeline.run();
-      fail("Run should thrown an exception");
-    } catch (RuntimeException e) {
-      assertNotNull(e.getCause());
-
-      // TODO: remove the version check (and the setup and teardown methods) when we no
-      // longer support Spark 1.3 or 1.4
-      String version = SparkContextFactory.getSparkContext(options.getSparkMaster(), options.getAppName()).version();
-      if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) {
-        assertTrue(e.getCause() instanceof UserException);
-      }
-    }
-  }
-
-  @Before
-  public void setup() {
-    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "true");
-  }
-
-  @After
-  public void teardown() {
-    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "false");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
index c7dc400..e32b39a 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
@@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.*;
 import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
 import org.junit.Test;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
deleted file mode 100644
index 23416d7..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestSparkPipelineOptionsFactory {
-  @Test
-  public void testDefaultCreateMethod() {
-    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
-    Assert.assertEquals("local[1]", actualOptions.getSparkMaster());
-  }
-
-  @Test
-  public void testSettingCustomOptions() {
-    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
-    actualOptions.setSparkMaster("spark://207.184.161.138:7077");
-    Assert.assertEquals("spark://207.184.161.138:7077", actualOptions.getSparkMaster());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
deleted file mode 100644
index ed58c77..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.api.client.repackaged.com.google.common.base.Joiner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Charsets;
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A test for the transforms registered in TransformTranslator.
- * Builds a regular Dataflow pipeline with each of the mapped
- * transforms, and makes sure that they work when the pipeline is
- * executed in Spark.
- */
-public class TransformTranslatorTest {
-
-  @Rule
-  public TestName name = new TestName();
-
-  private DirectPipelineRunner directRunner;
-  private SparkPipelineRunner sparkRunner;
-  private String testDataDirName;
-
-  @Before public void init() throws IOException {
-    sparkRunner = SparkPipelineRunner.create();
-    directRunner = DirectPipelineRunner.createForTest();
-    testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName())
-        + File.separator;
-    FileUtils.deleteDirectory(new File(testDataDirName));
-    new File(testDataDirName).mkdirs();
-  }
-
-  /**
-   * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline
-   * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
-   * transforms. Finally it makes sure that the results are the same for both runs.
-   */
-  @Test
-  public void testTextIOReadAndWriteTransforms() throws IOException {
-    String directOut = runPipeline("direct", directRunner);
-    String sparkOut = runPipeline("spark", sparkRunner);
-
-    List<String> directOutput =
-        Files.readAllLines(Paths.get(directOut + "-00000-of-00001"), Charsets.UTF_8);
-
-    List<String> sparkOutput =
-        Files.readAllLines(Paths.get(sparkOut + "-00000-of-00001"), Charsets.UTF_8);
-
-    // sort output to get a stable result (PCollections are not ordered)
-    Collections.sort(directOutput);
-    Collections.sort(sparkOutput);
-
-    Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray());
-  }
-
-  private String runPipeline(String name, PipelineRunner<?> runner) {
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out_" + name);
-    PCollection<String> lines =  p.apply(TextIO.Read.from("src/test/resources/test_text.txt"));
-    lines.apply(TextIO.Write.to(outFile));
-    runner.run(p);
-    return outFile;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
deleted file mode 100644
index 77409a0..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.joda.time.Duration;
-import org.junit.Test;
-
-public class WindowedWordCountTest {
-  private static final String[] WORDS_ARRAY = {
-          "hi there", "hi", "hi sue bob",
-          "hi sue", "", "bob hi"};
-  private static final Long[] TIMESTAMPS_ARRAY = {
-          60000L, 60000L, 60000L,
-          120000L, 120000L, 120000L};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-  private static final List<Long> TIMESTAMPS = Arrays.asList(TIMESTAMPS_ARRAY);
-  private static final List<String> EXPECTED_COUNT_SET =
-          ImmutableList.of("hi: 3", "there: 1", "sue: 1", "bob: 1",
-                  "hi: 2", "sue: 1", "bob: 1");
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    PCollection<String> inputWords = p.apply(Create.timestamped(WORDS, TIMESTAMPS))
-            .setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedWords = inputWords
-            .apply(Window.<String>into(FixedWindows.of(Duration.standardMinutes(1))));
-
-    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
index 5609e88..9edf41c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
@@ -30,7 +30,7 @@ import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.SparkPipelineOptionsFactory;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
deleted file mode 100644
index a7b9f28..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-
-import org.apache.beam.runners.spark.io.CreateStream;
-import org.apache.beam.runners.spark.EvaluationResult;
-import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.apache.beam.runners.spark.streaming.utils.DataflowAssertStreaming;
-
-import org.joda.time.Duration;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Test Flatten (union) implementation for streaming.
- */
-public class FlattenStreamingTest {
-
-  private static final String[] WORDS_ARRAY_1 = {
-      "one", "two", "three", "four"};
-  private static final List<Iterable<String>> WORDS_QUEUE_1 =
-      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_1));
-  private static final String[] WORDS_ARRAY_2 = {
-          "five", "six", "seven", "eight"};
-  private static final List<Iterable<String>> WORDS_QUEUE_2 =
-          Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_2));
-  private static final String[] EXPECTED_UNION = {
-          "one", "two", "three", "four", "five", "six", "seven", "eight"};
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @Test
-  public void testRun() throws Exception {
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<String> w1 =
-            p.apply(CreateStream.fromQueue(WORDS_QUEUE_1)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedW1 =
-            w1.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-    PCollection<String> w2 =
-            p.apply(CreateStream.fromQueue(WORDS_QUEUE_2)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedW2 =
-            w2.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-    PCollectionList<String> list = PCollectionList.of(windowedW1).and(windowedW2);
-    PCollection<String> union = list.apply(Flatten.<String>pCollections());
-
-    DataflowAssert.thatIterable(union.apply(View.<String>asIterable()))
-            .containsInAnyOrder(EXPECTED_UNION);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-
-}


[10/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateHelper.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateHelper.java
deleted file mode 100644
index 56980a1..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateHelper.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardNumber;
-
-public final class ShardNameTemplateHelper {
-
-  private static final Logger LOG = LoggerFactory.getLogger(ShardNameTemplateHelper.class);
-
-  public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.fileoutputformat.prefix";
-  public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.fileoutputformat.template";
-  public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.fileoutputformat.suffix";
-
-  private ShardNameTemplateHelper() {
-  }
-
-  public static <K, V> Path getDefaultWorkFile(FileOutputFormat<K, V> format,
-      TaskAttemptContext context) throws IOException {
-    FileOutputCommitter committer =
-        (FileOutputCommitter) format.getOutputCommitter(context);
-    return new Path(committer.getWorkPath(), getOutputFile(context));
-  }
-
-  private static String getOutputFile(TaskAttemptContext context) {
-    TaskID taskId = context.getTaskAttemptID().getTaskID();
-    int partition = taskId.getId();
-
-    String filePrefix = context.getConfiguration().get(OUTPUT_FILE_PREFIX);
-    String fileTemplate = context.getConfiguration().get(OUTPUT_FILE_TEMPLATE);
-    String fileSuffix = context.getConfiguration().get(OUTPUT_FILE_SUFFIX);
-    return filePrefix + replaceShardNumber(fileTemplate, partition) + fileSuffix;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java
deleted file mode 100644
index d3e8c9b..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.serializer.KryoSerializer;
-
-final class SparkContextFactory {
-
-  /**
-   * If the property {@code dataflow.spark.test.reuseSparkContext} is set to
-   * {@code true} then the Spark context will be reused for dataflow pipelines.
-   * This property should only be enabled for tests.
-   */
-  static final String TEST_REUSE_SPARK_CONTEXT =
-      "dataflow.spark.test.reuseSparkContext";
-  private static JavaSparkContext sparkContext;
-  private static String sparkMaster;
-
-  private SparkContextFactory() {
-  }
-
-  static synchronized JavaSparkContext getSparkContext(String master, String appName) {
-    if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
-      if (sparkContext == null) {
-        sparkContext = createSparkContext(master, appName);
-        sparkMaster = master;
-      } else if (!master.equals(sparkMaster)) {
-        throw new IllegalArgumentException(String.format("Cannot reuse spark context " +
-                "with different spark master URL. Existing: %s, requested: %s.",
-            sparkMaster, master));
-      }
-      return sparkContext;
-    } else {
-      return createSparkContext(master, appName);
-    }
-  }
-
-  static synchronized void stopSparkContext(JavaSparkContext context) {
-    if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
-      context.stop();
-    }
-  }
-
-  private static JavaSparkContext createSparkContext(String master, String appName) {
-    SparkConf conf = new SparkConf();
-    conf.setMaster(master);
-    conf.setAppName(appName);
-    conf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
-    return new JavaSparkContext(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineEvaluator.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineEvaluator.java
deleted file mode 100644
index 6762180..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineEvaluator.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-/**
- * Pipeline {@link SparkPipelineRunner.Evaluator} for Spark.
- */
-public final class SparkPipelineEvaluator extends SparkPipelineRunner.Evaluator {
-
-  private final EvaluationContext ctxt;
-
-  public SparkPipelineEvaluator(EvaluationContext ctxt, SparkPipelineTranslator translator) {
-    super(translator);
-    this.ctxt = ctxt;
-  }
-
-  @Override
-  protected <PT extends PTransform<? super PInput, POutput>> void doVisitTransform(TransformTreeNode
-      node) {
-    @SuppressWarnings("unchecked")
-    PT transform = (PT) node.getTransform();
-    @SuppressWarnings("unchecked")
-    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
-    @SuppressWarnings("unchecked") TransformEvaluator<PT> evaluator =
-        (TransformEvaluator<PT>) translator.translate(transformClass);
-    LOG.info("Evaluating {}", transform);
-    AppliedPTransform<PInput, POutput, PT> appliedTransform =
-        AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform);
-    ctxt.setCurrentTransform(appliedTransform);
-    evaluator.evaluate(transform, ctxt);
-    ctxt.setCurrentTransform(null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java
deleted file mode 100644
index e96162e..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-
-public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions,
-                                              ApplicationNameOptions {
-  @Description("The url of the spark master to connect to, (e.g. spark://host:port, local[4]).")
-  @Default.String("local[1]")
-  String getSparkMaster();
-
-  void setSparkMaster(String master);
-
-  @Override
-  @Default.Boolean(false)
-  boolean isStreaming();
-
-  @Override
-  @Default.String("spark dataflow pipeline job")
-  String getAppName();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java
deleted file mode 100644
index 89cd030..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-
-public final class SparkPipelineOptionsFactory {
-  private SparkPipelineOptionsFactory() {
-  }
-
-  public static SparkPipelineOptions create() {
-    return PipelineOptionsFactory.as(SparkPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsRegistrar.java
deleted file mode 100644
index 21fe693..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsRegistrar.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
-  @Override
-  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
deleted file mode 100644
index a9c2d86..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import org.apache.spark.SparkException;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptions;
-import com.cloudera.dataflow.spark.streaming.StreamingEvaluationContext;
-import com.cloudera.dataflow.spark.streaming.StreamingTransformTranslator;
-import com.cloudera.dataflow.spark.streaming.StreamingWindowPipelineDetector;
-
-/**
- * The SparkPipelineRunner translate operations defined on a pipeline to a representation
- * executable by Spark, and then submitting the job to Spark to be executed. If we wanted to run
- * a dataflow pipeline with the default options of a single threaded spark instance in local mode,
- * we would do the following:
- *
- * {@code
- * Pipeline p = [logic for pipeline creation]
- * EvaluationResult result = SparkPipelineRunner.create().run(p);
- * }
- *
- * To create a pipeline runner to run against a different spark cluster, with a custom master url
- * we would do the following:
- *
- * {@code
- * Pipeline p = [logic for pipeline creation]
- * SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
- * options.setSparkMaster("spark://host:port");
- * EvaluationResult result = SparkPipelineRunner.create(options).run(p);
- * }
- *
- * To create a Spark streaming pipeline runner use {@link SparkStreamingPipelineOptions}
- */
-public final class SparkPipelineRunner extends PipelineRunner<EvaluationResult> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(SparkPipelineRunner.class);
-  /**
-   * Options used in this pipeline runner.
-   */
-  private final SparkPipelineOptions mOptions;
-
-  /**
-   * Creates and returns a new SparkPipelineRunner with default options. In particular, against a
-   * spark instance running in local mode.
-   *
-   * @return A pipeline runner with default options.
-   */
-  public static SparkPipelineRunner create() {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    return new SparkPipelineRunner(options);
-  }
-
-  /**
-   * Creates and returns a new SparkPipelineRunner with specified options.
-   *
-   * @param options The SparkPipelineOptions to use when executing the job.
-   * @return A pipeline runner that will execute with specified options.
-   */
-  public static SparkPipelineRunner create(SparkPipelineOptions options) {
-    return new SparkPipelineRunner(options);
-  }
-
-  /**
-   * Creates and returns a new SparkPipelineRunner with specified options.
-   *
-   * @param options The PipelineOptions to use when executing the job.
-   * @return A pipeline runner that will execute with specified options.
-   */
-  public static SparkPipelineRunner fromOptions(PipelineOptions options) {
-    SparkPipelineOptions sparkOptions =
-        PipelineOptionsValidator.validate(SparkPipelineOptions.class, options);
-    return new SparkPipelineRunner(sparkOptions);
-  }
-
-  /**
-   * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single
-   * thread.
-   */
-  private SparkPipelineRunner(SparkPipelineOptions options) {
-    mOptions = options;
-  }
-
-
-  @Override
-  public EvaluationResult run(Pipeline pipeline) {
-    try {
-      // validate streaming configuration
-      if (mOptions.isStreaming() && !(mOptions instanceof SparkStreamingPipelineOptions)) {
-        throw new RuntimeException("A streaming job must be configured with " +
-            SparkStreamingPipelineOptions.class.getSimpleName() + ", found " +
-            mOptions.getClass().getSimpleName());
-      }
-      LOG.info("Executing pipeline using the SparkPipelineRunner.");
-      JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions
-              .getSparkMaster(), mOptions.getAppName());
-
-      if (mOptions.isStreaming()) {
-        SparkPipelineTranslator translator =
-                new StreamingTransformTranslator.Translator(new TransformTranslator.Translator());
-        // if streaming - fixed window should be defined on all UNBOUNDED inputs
-        StreamingWindowPipelineDetector streamingWindowPipelineDetector =
-            new StreamingWindowPipelineDetector(translator);
-        pipeline.traverseTopologically(streamingWindowPipelineDetector);
-        if (!streamingWindowPipelineDetector.isWindowing()) {
-          throw new IllegalStateException("Spark streaming pipeline must be windowed!");
-        }
-
-        Duration batchInterval = streamingWindowPipelineDetector.getBatchDuration();
-        LOG.info("Setting Spark streaming batchInterval to {} msec", batchInterval.milliseconds());
-        EvaluationContext ctxt = createStreamingEvaluationContext(jsc, pipeline, batchInterval);
-
-        pipeline.traverseTopologically(new SparkPipelineEvaluator(ctxt, translator));
-        ctxt.computeOutputs();
-
-        LOG.info("Streaming pipeline construction complete. Starting execution..");
-        ((StreamingEvaluationContext) ctxt).getStreamingContext().start();
-
-        return ctxt;
-      } else {
-        EvaluationContext ctxt = new EvaluationContext(jsc, pipeline);
-        SparkPipelineTranslator translator = new TransformTranslator.Translator();
-        pipeline.traverseTopologically(new SparkPipelineEvaluator(ctxt, translator));
-        ctxt.computeOutputs();
-
-        LOG.info("Pipeline execution complete.");
-
-        return ctxt;
-      }
-    } catch (Exception e) {
-      // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler
-      // won't let you catch something that is not declared, so we can't catch
-      // SparkException here. Instead we do an instanceof check.
-      // Then we find the cause by seeing if it's a user exception (wrapped by our
-      // SparkProcessException), or just use the SparkException cause.
-      if (e instanceof SparkException && e.getCause() != null) {
-        if (e.getCause() instanceof SparkProcessContext.SparkProcessException &&
-                e.getCause().getCause() != null) {
-          throw new RuntimeException(e.getCause().getCause());
-        } else {
-          throw new RuntimeException(e.getCause());
-        }
-      }
-      // otherwise just wrap in a RuntimeException
-      throw new RuntimeException(e);
-    }
-  }
-
-  private EvaluationContext
-      createStreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
-      Duration batchDuration) {
-    SparkStreamingPipelineOptions streamingOptions = (SparkStreamingPipelineOptions) mOptions;
-    JavaStreamingContext jssc = new JavaStreamingContext(jsc, batchDuration);
-    return new StreamingEvaluationContext(jsc, pipeline, jssc, streamingOptions.getTimeout());
-  }
-
-  public abstract static class Evaluator implements Pipeline.PipelineVisitor {
-    protected static final Logger LOG = LoggerFactory.getLogger(Evaluator.class);
-
-    protected final SparkPipelineTranslator translator;
-
-    protected Evaluator(SparkPipelineTranslator translator) {
-      this.translator = translator;
-    }
-
-    // Set upon entering a composite node which can be directly mapped to a single
-    // TransformEvaluator.
-    private TransformTreeNode currentTranslatedCompositeNode;
-
-    /**
-     * If true, we're currently inside a subtree of a composite node which directly maps to a
-     * single
-     * TransformEvaluator; children nodes are ignored, and upon post-visiting the translated
-     * composite node, the associated TransformEvaluator will be visited.
-     */
-    private boolean inTranslatedCompositeNode() {
-      return currentTranslatedCompositeNode != null;
-    }
-
-    @Override
-    public void enterCompositeTransform(TransformTreeNode node) {
-      if (!inTranslatedCompositeNode() && node.getTransform() != null) {
-        @SuppressWarnings("unchecked")
-        Class<PTransform<?, ?>> transformClass =
-            (Class<PTransform<?, ?>>) node.getTransform().getClass();
-        if (translator.hasTranslation(transformClass)) {
-          LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName());
-          LOG.debug("Composite transform class: '{}'", transformClass);
-          currentTranslatedCompositeNode = node;
-        }
-      }
-    }
-
-    @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
-      // NB: We depend on enterCompositeTransform and leaveCompositeTransform providing 'node'
-      // objects for which Object.equals() returns true iff they are the same logical node
-      // within the tree.
-      if (inTranslatedCompositeNode() && node.equals(currentTranslatedCompositeNode)) {
-        LOG.info("Post-visiting directly-translatable composite transform: '{}'",
-                node.getFullName());
-        doVisitTransform(node);
-        currentTranslatedCompositeNode = null;
-      }
-    }
-
-    @Override
-    public void visitTransform(TransformTreeNode node) {
-      if (inTranslatedCompositeNode()) {
-        LOG.info("Skipping '{}'; already in composite transform.", node.getFullName());
-        return;
-      }
-      doVisitTransform(node);
-    }
-
-    protected abstract <PT extends PTransform<? super PInput, POutput>> void
-        doVisitTransform(TransformTreeNode node);
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunnerRegistrar.java
deleted file mode 100644
index 5bdd322..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunnerRegistrar.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkPipelineRunnerRegistrar implements PipelineRunnerRegistrar {
-  @Override
-  public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-    return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java
deleted file mode 100644
index d90363f..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-/**
- * Translator to support translation between Dataflow transformations and Spark transformations.
- */
-public interface SparkPipelineTranslator {
-
-  boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz);
-
-  <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java
deleted file mode 100644
index 73cec25..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class SparkProcessContext<I, O, V> extends DoFn<I, O>.ProcessContext {
-
-  private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class);
-
-  private final DoFn<I, O> fn;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  protected WindowedValue<I> windowedValue;
-
-  SparkProcessContext(DoFn<I, O> fn,
-      SparkRuntimeContext runtime,
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    fn.super();
-    this.fn = fn;
-    this.mRuntimeContext = runtime;
-    this.mSideInputs = sideInputs;
-  }
-
-  void setup() {
-    setupDelegateAggregators();
-  }
-
-  @Override
-  public PipelineOptions getPipelineOptions() {
-    return mRuntimeContext.getPipelineOptions();
-  }
-
-  @Override
-  public <T> T sideInput(PCollectionView<T> view) {
-    @SuppressWarnings("unchecked")
-    BroadcastHelper<Iterable<WindowedValue<?>>> broadcastHelper =
-        (BroadcastHelper<Iterable<WindowedValue<?>>>) mSideInputs.get(view.getTagInternal());
-    Iterable<WindowedValue<?>> contents = broadcastHelper.getValue();
-    return view.fromIterableInternal(contents);
-  }
-
-  @Override
-  public abstract void output(O output);
-
-  public abstract void output(WindowedValue<O> output);
-
-  @Override
-  public <T> void sideOutput(TupleTag<T> tupleTag, T t) {
-    String message = "sideOutput is an unsupported operation for doFunctions, use a " +
-        "MultiDoFunction instead.";
-    LOG.warn(message);
-    throw new UnsupportedOperationException(message);
-  }
-
-  @Override
-  public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
-    String message =
-        "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " +
-            "MultiDoFunction instead.";
-    LOG.warn(message);
-    throw new UnsupportedOperationException(message);
-  }
-
-  @Override
-  public <AI, AO> Aggregator<AI, AO> createAggregatorInternal(
-      String named,
-      Combine.CombineFn<AI, ?, AO> combineFn) {
-    return mRuntimeContext.createAggregator(named, combineFn);
-  }
-
-  @Override
-  public I element() {
-    return windowedValue.getValue();
-  }
-
-  @Override
-  public void outputWithTimestamp(O output, Instant timestamp) {
-    output(WindowedValue.of(output, timestamp,
-        windowedValue.getWindows(), windowedValue.getPane()));
-  }
-
-  @Override
-  public Instant timestamp() {
-    return windowedValue.getTimestamp();
-  }
-
-  @Override
-  public BoundedWindow window() {
-    if (!(fn instanceof DoFn.RequiresWindowAccess)) {
-      throw new UnsupportedOperationException(
-          "window() is only available in the context of a DoFn marked as RequiresWindow.");
-    }
-    return Iterables.getOnlyElement(windowedValue.getWindows());
-  }
-
-  @Override
-  public PaneInfo pane() {
-    return windowedValue.getPane();
-  }
-
-  @Override
-  public WindowingInternals<I, O> windowingInternals() {
-    return new WindowingInternals<I, O>() {
-
-      @Override
-      public Collection<? extends BoundedWindow> windows() {
-        return windowedValue.getWindows();
-      }
-
-      @Override
-      public void outputWindowedValue(O output, Instant timestamp, Collection<?
-          extends BoundedWindow> windows, PaneInfo paneInfo) {
-        output(WindowedValue.of(output, timestamp, windows, paneInfo));
-      }
-
-      @Override
-      public StateInternals stateInternals() {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#stateInternals() is not yet supported.");
-      }
-
-      @Override
-      public TimerInternals timerInternals() {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#timerInternals() is not yet supported.");
-      }
-
-      @Override
-      public PaneInfo pane() {
-        return windowedValue.getPane();
-      }
-
-      @Override
-      public <T> void writePCollectionViewData(TupleTag<?> tag,
-          Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#writePCollectionViewData() is not yet supported.");
-      }
-    };
-  }
-
-  protected abstract void clearOutput();
-  protected abstract Iterator<V> getOutputIterator();
-
-  protected Iterable<V> getOutputIterable(final Iterator<WindowedValue<I>> iter,
-      final DoFn<I, O> doFn) {
-    return new Iterable<V>() {
-      @Override
-      public Iterator<V> iterator() {
-        return new ProcCtxtIterator(iter, doFn);
-      }
-    };
-  }
-
-  private class ProcCtxtIterator extends AbstractIterator<V> {
-
-    private final Iterator<WindowedValue<I>> inputIterator;
-    private final DoFn<I, O> doFn;
-    private Iterator<V> outputIterator;
-    private boolean calledFinish;
-
-    ProcCtxtIterator(Iterator<WindowedValue<I>> iterator, DoFn<I, O> doFn) {
-      this.inputIterator = iterator;
-      this.doFn = doFn;
-      this.outputIterator = getOutputIterator();
-    }
-
-    @Override
-    protected V computeNext() {
-      // Process each element from the (input) iterator, which produces, zero, one or more
-      // output elements (of type V) in the output iterator. Note that the output
-      // collection (and iterator) is reset between each call to processElement, so the
-      // collection only holds the output values for each call to processElement, rather
-      // than for the whole partition (which would use too much memory).
-      while (true) {
-        if (outputIterator.hasNext()) {
-          return outputIterator.next();
-        } else if (inputIterator.hasNext()) {
-          clearOutput();
-          windowedValue = inputIterator.next();
-          try {
-            doFn.processElement(SparkProcessContext.this);
-          } catch (Exception e) {
-            throw new SparkProcessException(e);
-          }
-          outputIterator = getOutputIterator();
-        } else {
-          // no more input to consume, but finishBundle can produce more output
-          if (!calledFinish) {
-            clearOutput();
-            try {
-              calledFinish = true;
-              doFn.finishBundle(SparkProcessContext.this);
-            } catch (Exception e) {
-              throw new SparkProcessException(e);
-            }
-            outputIterator = getOutputIterator();
-            continue; // try to consume outputIterator from start of loop
-          }
-          return endOfData();
-        }
-      }
-    }
-  }
-
-  static class SparkProcessException extends RuntimeException {
-    SparkProcessException(Throwable t) {
-      super(t);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java
deleted file mode 100644
index ec590a9..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.collect.ImmutableList;
-import org.apache.spark.Accumulator;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import com.cloudera.dataflow.spark.aggregators.AggAccumParam;
-import com.cloudera.dataflow.spark.aggregators.NamedAggregators;
-
-/**
- * The SparkRuntimeContext allows us to define useful features on the client side before our
- * data flow program is launched.
- */
-public class SparkRuntimeContext implements Serializable {
-  /**
-   * An accumulator that is a map from names to aggregators.
-   */
-  private final Accumulator<NamedAggregators> accum;
-
-  private final String serializedPipelineOptions;
-
-  /**
-   * Map fo names to dataflow aggregators.
-   */
-  private final Map<String, Aggregator<?, ?>> aggregators = new HashMap<>();
-  private transient CoderRegistry coderRegistry;
-
-  SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) {
-    this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam());
-    this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions());
-  }
-
-  private static String serializePipelineOptions(PipelineOptions pipelineOptions) {
-    try {
-      return new ObjectMapper().writeValueAsString(pipelineOptions);
-    } catch (JsonProcessingException e) {
-      throw new IllegalStateException("Failed to serialize the pipeline options.", e);
-    }
-  }
-
-  private static PipelineOptions deserializePipelineOptions(String serializedPipelineOptions) {
-    try {
-      return new ObjectMapper().readValue(serializedPipelineOptions, PipelineOptions.class);
-    } catch (IOException e) {
-      throw new IllegalStateException("Failed to deserialize the pipeline options.", e);
-    }
-  }
-
-  /**
-   * Retrieves corresponding value of an aggregator.
-   *
-   * @param aggregatorName Name of the aggregator to retrieve the value of.
-   * @param typeClass      Type class of value to be retrieved.
-   * @param <T>            Type of object to be returned.
-   * @return The value of the aggregator.
-   */
-  public <T> T getAggregatorValue(String aggregatorName, Class<T> typeClass) {
-    return accum.value().getValue(aggregatorName, typeClass);
-  }
-
-  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
-    @SuppressWarnings("unchecked")
-    Class<T> aggValueClass = (Class<T>) aggregator.getCombineFn().getOutputType().getRawType();
-    final T aggregatorValue = getAggregatorValue(aggregator.getName(), aggValueClass);
-    return new AggregatorValues<T>() {
-      @Override
-      public Collection<T> getValues() {
-        return ImmutableList.of(aggregatorValue);
-      }
-
-      @Override
-      public Map<String, T> getValuesAtSteps() {
-        throw new UnsupportedOperationException("getValuesAtSteps is not supported.");
-      }
-    };
-  }
-
-  public synchronized PipelineOptions getPipelineOptions() {
-    return deserializePipelineOptions(serializedPipelineOptions);
-  }
-
-  /**
-   * Creates and aggregator and associates it with the specified name.
-   *
-   * @param named     Name of aggregator.
-   * @param combineFn Combine function used in aggregation.
-   * @param <IN>      Type of inputs to aggregator.
-   * @param <INTER>   Intermediate data type
-   * @param <OUT>     Type of aggregator outputs.
-   * @return Specified aggregator
-   */
-  public synchronized <IN, INTER, OUT> Aggregator<IN, OUT> createAggregator(
-      String named,
-      Combine.CombineFn<? super IN, INTER, OUT> combineFn) {
-    @SuppressWarnings("unchecked")
-    Aggregator<IN, OUT> aggregator = (Aggregator<IN, OUT>) aggregators.get(named);
-    if (aggregator == null) {
-      @SuppressWarnings("unchecked")
-      NamedAggregators.CombineFunctionState<IN, INTER, OUT> state =
-          new NamedAggregators.CombineFunctionState<>(
-              (Combine.CombineFn<IN, INTER, OUT>) combineFn,
-              (Coder<IN>) getCoder(combineFn),
-              this);
-      accum.add(new NamedAggregators(named, state));
-      aggregator = new SparkAggregator<>(named, state);
-      aggregators.put(named, aggregator);
-    }
-    return aggregator;
-  }
-
-  public CoderRegistry getCoderRegistry() {
-    if (coderRegistry == null) {
-      coderRegistry = new CoderRegistry();
-      coderRegistry.registerStandardCoders();
-    }
-    return coderRegistry;
-  }
-
-  private Coder<?> getCoder(Combine.CombineFn<?, ?, ?> combiner) {
-    try {
-      if (combiner.getClass() == Sum.SumIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Sum.SumLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Sum.SumDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else if (combiner.getClass() == Min.MinIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Min.MinLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Min.MinDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else if (combiner.getClass() == Max.MaxIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Max.MaxLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Max.MaxDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else {
-        throw new IllegalArgumentException("unsupported combiner in Aggregator: "
-            + combiner.getClass().getName());
-      }
-    } catch (CannotProvideCoderException e) {
-      throw new IllegalStateException("Could not determine default coder for combiner", e);
-    }
-  }
-
-  /**
-   * Initialize spark aggregators exactly once.
-   *
-   * @param <IN> Type of element fed in to aggregator.
-   */
-  private static class SparkAggregator<IN, OUT> implements Aggregator<IN, OUT>, Serializable {
-    private final String name;
-    private final NamedAggregators.State<IN, ?, OUT> state;
-
-    SparkAggregator(String name, NamedAggregators.State<IN, ?, OUT> state) {
-      this.name = name;
-      this.state = state;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public void addValue(IN elem) {
-      state.update(elem);
-    }
-
-    @Override
-    public Combine.CombineFn<IN, ?, OUT> getCombineFn() {
-      return state.getCombineFn();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedAvroKeyOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedAvroKeyOutputFormat.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedAvroKeyOutputFormat.java
deleted file mode 100644
index ef24137..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedAvroKeyOutputFormat.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.avro.mapreduce.AvroKeyOutputFormat;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-public class TemplatedAvroKeyOutputFormat<T> extends AvroKeyOutputFormat<T>
-    implements ShardNameTemplateAware {
-
-  @Override
-  public void checkOutputSpecs(JobContext job) {
-    // don't fail if the output already exists
-  }
-
-  @Override
-  protected OutputStream getAvroFileOutputStream(TaskAttemptContext context) throws IOException {
-    Path path = ShardNameTemplateHelper.getDefaultWorkFile(this, context);
-    return path.getFileSystem(context.getConfiguration()).create(path);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedSequenceFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedSequenceFileOutputFormat.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedSequenceFileOutputFormat.java
deleted file mode 100644
index 3ab07b5..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedSequenceFileOutputFormat.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-public class TemplatedSequenceFileOutputFormat<K, V> extends SequenceFileOutputFormat<K, V>
-    implements ShardNameTemplateAware {
-
-  @Override
-  public void checkOutputSpecs(JobContext job) {
-    // don't fail if the output already exists
-  }
-
-  @Override
-  public Path getDefaultWorkFile(TaskAttemptContext context,
-      String extension) throws IOException {
-    // note that the passed-in extension is ignored since it comes from the template
-    return ShardNameTemplateHelper.getDefaultWorkFile(this, context);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java
deleted file mode 100644
index a8e218d..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-
-public class TemplatedTextOutputFormat<K, V> extends TextOutputFormat<K, V>
-    implements ShardNameTemplateAware {
-
-  @Override
-  public void checkOutputSpecs(JobContext job) {
-    // don't fail if the output already exists
-  }
-
-  @Override
-  public Path getDefaultWorkFile(TaskAttemptContext context,
-      String extension) throws IOException {
-    // note that the passed-in extension is ignored since it comes from the template
-    return ShardNameTemplateHelper.getDefaultWorkFile(this, context);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java
deleted file mode 100644
index 52842d5..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.Serializable;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-public interface TransformEvaluator<PT extends PTransform<?, ?>> extends Serializable {
-  void evaluate(PT transform, EvaluationContext context);
-}


[03/23] incubator-beam git commit: [BEAM-11] set coder for pipeline input

Posted by am...@apache.org.
[BEAM-11] set coder for pipeline input


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

Branch: refs/heads/master
Commit: c5c7df603289b1d207308da16546dd56e9b9b6d9
Parents: 41c4ca6
Author: Sela <an...@paypal.com>
Authored: Sat Mar 12 17:26:34 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:38:26 2016 +0200

----------------------------------------------------------------------
 .../spark/io/hadoop/HadoopFileFormatPipelineTest.java    |  5 ++++-
 .../beam/runners/spark/streaming/KafkaStreamingTest.java | 11 +++++++----
 2 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c5c7df60/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
index 7a9be8b..abe1119 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
@@ -16,11 +16,13 @@
 package org.apache.beam.runners.spark.io.hadoop;
 
 import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
 import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
 import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.cloud.dataflow.sdk.values.PCollection;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.coders.WritableCoder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -66,7 +68,8 @@ public class HadoopFileFormatPipelineTest {
         (Class<? extends FileInputFormat<IntWritable, Text>>) (Class<?>) SequenceFileInputFormat.class;
     HadoopIO.Read.Bound<IntWritable,Text> read =
         HadoopIO.Read.from(inputFile.getAbsolutePath(), inputFormatClass, IntWritable.class, Text.class);
-    PCollection<KV<IntWritable, Text>> input = p.apply(read);
+    PCollection<KV<IntWritable, Text>> input = p.apply(read)
+        .setCoder(KvCoder.of(WritableCoder.of(IntWritable.class), WritableCoder.of(Text.class)));
     @SuppressWarnings("unchecked")
     Class<? extends FileOutputFormat<IntWritable, Text>> outputFormatClass =
         (Class<? extends FileOutputFormat<IntWritable, Text>>) (Class<?>) TemplatedSequenceFileOutputFormat.class;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c5c7df60/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
index e9e685b..ff1e11c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
@@ -15,6 +15,8 @@
 package org.apache.beam.runners.spark.streaming;
 
 import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
 import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
 import com.google.cloud.dataflow.sdk.transforms.DoFn;
 import com.google.cloud.dataflow.sdk.transforms.ParDo;
@@ -79,7 +81,7 @@ public class KafkaStreamingTest {
     producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
     Serializer<String> stringSerializer = new StringSerializer();
     try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
-            new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
+        new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
       for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
         kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
       }
@@ -96,13 +98,14 @@ public class KafkaStreamingTest {
     Pipeline p = Pipeline.create(options);
 
     Map<String, String> kafkaParams = ImmutableMap.of(
-            "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
-            "auto.offset.reset", "smallest"
+        "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
+        "auto.offset.reset", "smallest"
     );
 
     PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
         StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
-        kafkaParams));
+        kafkaParams))
+        .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
     PCollection<KV<String, String>> windowedWords = kafkaInput
         .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
 


[21/23] incubator-beam git commit: [BEAM-11] Replaced license headers to ASF license

Posted by am...@apache.org.
[BEAM-11] Replaced license headers to ASF license


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

Branch: refs/heads/master
Commit: 6ef36411886b4cbd053532692bdac019b49f809a
Parents: 95ebf89
Author: Sela <an...@paypal.com>
Authored: Mon Mar 14 19:09:14 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:40:18 2016 +0200

----------------------------------------------------------------------
 runners/spark/build-resources/checkstyle.xml    | 27 +++++++++++---------
 runners/spark/build-resources/header-file.txt   | 23 +++++++++--------
 .../apache/beam/runners/spark/DoFnFunction.java | 23 +++++++++--------
 .../beam/runners/spark/EvaluationContext.java   | 23 +++++++++--------
 .../beam/runners/spark/EvaluationResult.java    | 23 +++++++++--------
 .../beam/runners/spark/MultiDoFnFunction.java   | 23 +++++++++--------
 .../beam/runners/spark/SparkContextFactory.java | 23 +++++++++--------
 .../runners/spark/SparkPipelineEvaluator.java   | 23 +++++++++--------
 .../runners/spark/SparkPipelineOptions.java     | 23 +++++++++--------
 .../spark/SparkPipelineOptionsFactory.java      | 23 +++++++++--------
 .../spark/SparkPipelineOptionsRegistrar.java    | 23 +++++++++--------
 .../beam/runners/spark/SparkPipelineRunner.java | 23 +++++++++--------
 .../spark/SparkPipelineRunnerRegistrar.java     | 23 +++++++++--------
 .../runners/spark/SparkPipelineTranslator.java  | 23 +++++++++--------
 .../beam/runners/spark/SparkProcessContext.java | 23 +++++++++--------
 .../beam/runners/spark/SparkRuntimeContext.java | 23 +++++++++--------
 .../beam/runners/spark/TransformEvaluator.java  | 23 +++++++++--------
 .../beam/runners/spark/TransformTranslator.java | 23 +++++++++--------
 .../beam/runners/spark/WindowingHelpers.java    | 23 +++++++++--------
 .../spark/aggregators/AggAccumParam.java        | 23 +++++++++--------
 .../spark/aggregators/NamedAggregators.java     | 23 +++++++++--------
 .../beam/runners/spark/coders/CoderHelpers.java | 23 +++++++++--------
 .../runners/spark/coders/NullWritableCoder.java | 23 +++++++++--------
 .../runners/spark/coders/WritableCoder.java     | 23 +++++++++--------
 .../apache/beam/runners/spark/io/ConsoleIO.java | 23 +++++++++--------
 .../beam/runners/spark/io/CreateStream.java     | 23 +++++++++--------
 .../apache/beam/runners/spark/io/KafkaIO.java   | 23 +++++++++--------
 .../beam/runners/spark/io/hadoop/HadoopIO.java  | 23 +++++++++--------
 .../spark/io/hadoop/ShardNameBuilder.java       | 23 +++++++++--------
 .../spark/io/hadoop/ShardNameTemplateAware.java | 23 +++++++++--------
 .../io/hadoop/ShardNameTemplateHelper.java      | 23 +++++++++--------
 .../io/hadoop/TemplatedAvroKeyOutputFormat.java | 23 +++++++++--------
 .../TemplatedSequenceFileOutputFormat.java      | 23 +++++++++--------
 .../io/hadoop/TemplatedTextOutputFormat.java    | 23 +++++++++--------
 .../SparkStreamingPipelineOptions.java          | 23 +++++++++--------
 .../SparkStreamingPipelineOptionsFactory.java   | 23 +++++++++--------
 .../SparkStreamingPipelineOptionsRegistrar.java | 23 +++++++++--------
 .../streaming/StreamingEvaluationContext.java   | 23 +++++++++--------
 .../streaming/StreamingTransformTranslator.java | 23 +++++++++--------
 .../StreamingWindowPipelineDetector.java        | 23 +++++++++--------
 .../runners/spark/util/BroadcastHelper.java     | 23 +++++++++--------
 .../beam/runners/spark/util/ByteArray.java      | 23 +++++++++--------
 .../beam/runners/spark/CombineGloballyTest.java | 23 +++++++++--------
 .../beam/runners/spark/CombinePerKeyTest.java   | 23 +++++++++--------
 .../apache/beam/runners/spark/DeDupTest.java    | 23 +++++++++--------
 .../beam/runners/spark/DoFnOutputTest.java      | 23 +++++++++--------
 .../beam/runners/spark/EmptyInputTest.java      | 23 +++++++++--------
 .../runners/spark/MultiOutputWordCountTest.java | 23 +++++++++--------
 .../beam/runners/spark/SerializationTest.java   | 23 +++++++++--------
 .../beam/runners/spark/SideEffectsTest.java     | 23 +++++++++--------
 .../beam/runners/spark/SimpleWordCountTest.java | 23 +++++++++--------
 .../spark/TestSparkPipelineOptionsFactory.java  | 23 +++++++++--------
 .../apache/beam/runners/spark/TfIdfTest.java    | 23 +++++++++--------
 .../runners/spark/TransformTranslatorTest.java  | 23 +++++++++--------
 .../runners/spark/WindowedWordCountTest.java    | 23 +++++++++--------
 .../runners/spark/coders/WritableCoderTest.java | 23 +++++++++--------
 .../beam/runners/spark/io/AvroPipelineTest.java | 23 +++++++++--------
 .../beam/runners/spark/io/NumShardsTest.java    | 23 +++++++++--------
 .../io/hadoop/HadoopFileFormatPipelineTest.java | 23 +++++++++--------
 .../spark/io/hadoop/ShardNameBuilderTest.java   | 23 +++++++++--------
 .../spark/streaming/FlattenStreamingTest.java   | 23 +++++++++--------
 .../spark/streaming/KafkaStreamingTest.java     | 23 +++++++++--------
 .../streaming/SimpleStreamingWordCountTest.java | 23 +++++++++--------
 .../utils/DataflowAssertStreaming.java          | 23 +++++++++--------
 .../streaming/utils/EmbeddedKafkaCluster.java   | 23 +++++++++--------
 65 files changed, 847 insertions(+), 652 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/build-resources/checkstyle.xml
----------------------------------------------------------------------
diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml
index c5b884d..033d8ae 100644
--- a/runners/spark/build-resources/checkstyle.xml
+++ b/runners/spark/build-resources/checkstyle.xml
@@ -3,18 +3,21 @@
         "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
         "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
 <!--
-  Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
-
-  Cloudera, Inc. 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
-
-  This software 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.
+   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.
   -->
 <!--
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/build-resources/header-file.txt
----------------------------------------------------------------------
diff --git a/runners/spark/build-resources/header-file.txt b/runners/spark/build-resources/header-file.txt
index 6d81b4d..d551913 100644
--- a/runners/spark/build-resources/header-file.txt
+++ b/runners/spark/build-resources/header-file.txt
@@ -1,14 +1,17 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * 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.
  */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
index 6af829f..e5d4542 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
index 836987f..ad49528 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
index 4de97f6..fa4776d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
index 968825b..47433a6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
index 10b7369..4393a75 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
index 913e5a1..becf15a 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
index 1a5093b..a7f65cc 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
index 7b44ee4..9bff013 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
index 9f7f8c1..c68af64 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
index 429750d..b1a402f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
index 9a84370..7861685 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
index e45491a..2e38a07 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
index f4d2ca0..b3a720d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
index da48ad7..f0f9974 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
index 8aaceeb..4b4f81f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
index e64f89a..6a9dca0 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformTranslator.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
index 6b904f7..bc09d5b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/WindowingHelpers.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
index a82dbbe..a75aeb3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggAccumParam.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.aggregators;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
index 2747703..c3718f4 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.aggregators;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
index 7d75e7d..11ce447 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
index 5b77e97..08a38e7 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
index fa73753..1378977 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
index 2ee072a..681dc72 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
index c92f8bf..7033104 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
index 9798157..0714d38 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
index e8d2aa1..b9065fa 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.io.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
index 21c7985..9e3ba82 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
index fdee42b..a4fd6f0 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
index fd6f5da..44fe26c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
index 4feaff6..b755928 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
index 922b906..35b6163 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
index 1e53dce..8f0c0d2 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
index 17edba3..f96f4dd 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
index 822feb4..ae04ebe 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
index 2c5414d..256820c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
index 9d1d786..50c889c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
index c78c7fa..56d0dd9 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
index 6844011..9c58126 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.streaming;



[14/23] incubator-beam git commit: [BEAM-11] relocate Guava used by Dataflow (v19) since it conflicts with version used by Hadoop (v11)

Posted by am...@apache.org.
[BEAM-11] relocate Guava used by Dataflow (v19) since it conflicts with version used by Hadoop (v11)


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

Branch: refs/heads/master
Commit: 95ebf8905fcc5c7b2f8d68bf293d471c80524019
Parents: bde9933
Author: Sela <an...@paypal.com>
Authored: Mon Mar 14 18:50:37 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:40:18 2016 +0200

----------------------------------------------------------------------
 runners/spark/pom.xml | 28 ++++++++++++++--------------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/95ebf890/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index a060161..8d2704f 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -346,20 +346,20 @@ License.
                             <goals>
                                 <goal>shade</goal>
                             </goals>
-                            <!--<configuration>-->
-                                <!--<relocations>-->
-                                    <!--&lt;!&ndash; relocate Guava used by Dataflow (v18) since it conflicts with version used by Hadoop (v11) &ndash;&gt;-->
-                                    <!--<relocation>-->
-                                        <!--<pattern>com.google.common</pattern>-->
-                                        <!--<shadedPattern>com.cloudera.dataflow.spark.relocated.com.google.common</shadedPattern>-->
-                                    <!--</relocation>-->
-                                <!--</relocations>-->
-                                <!--<shadedArtifactAttached>true</shadedArtifactAttached>-->
-                                <!--<shadedClassifierName>spark-app</shadedClassifierName>-->
-                                <!--<transformers>-->
-                                    <!--<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />-->
-                                <!--</transformers>-->
-                            <!--</configuration>-->
+                            <configuration>
+                                <relocations>
+                                    <!-- relocate Guava used by Dataflow (v18) since it conflicts with version used by Hadoop (v11) -->
+                                    <relocation>
+                                        <pattern>com.google.common</pattern>
+                                        <shadedPattern>org.apache.beam.spark.relocated.com.google.common</shadedPattern>
+                                    </relocation>
+                                </relocations>
+                                <shadedArtifactAttached>true</shadedArtifactAttached>
+                                <shadedClassifierName>spark-app</shadedClassifierName>
+                                <transformers>
+                                    <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                </transformers>
+                            </configuration>
                         </execution>
                     </executions>
                 </plugin>


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

Posted by am...@apache.org.
This closes #42


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

Branch: refs/heads/master
Commit: a91e1150084808202e382f169eb273e5b52bbc97
Parents: 46412e5 b49e3c9
Author: Sela <an...@paypal.com>
Authored: Tue Mar 15 20:41:13 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:41:13 2016 +0200

----------------------------------------------------------------------
 README.md                                       |   7 +-
 runners/pom.xml                                 |   1 +
 runners/spark/.gitignore                        |  10 -
 runners/spark/.travis.yml                       |  22 -
 runners/spark/build-resources/checkstyle.xml    |  27 +-
 runners/spark/build-resources/header-file.txt   |  23 +-
 runners/spark/pom.xml                           | 246 +++---
 .../com/cloudera/dataflow/hadoop/HadoopIO.java  | 202 -----
 .../dataflow/hadoop/NullWritableCoder.java      |  71 --
 .../cloudera/dataflow/hadoop/WritableCoder.java | 120 ---
 .../com/cloudera/dataflow/io/ConsoleIO.java     |  60 --
 .../com/cloudera/dataflow/io/CreateStream.java  |  66 --
 .../java/com/cloudera/dataflow/io/KafkaIO.java  | 128 ---
 .../dataflow/spark/BroadcastHelper.java         | 121 ---
 .../com/cloudera/dataflow/spark/ByteArray.java  |  52 --
 .../cloudera/dataflow/spark/CoderHelpers.java   | 185 -----
 .../cloudera/dataflow/spark/DoFnFunction.java   |  93 ---
 .../dataflow/spark/EvaluationContext.java       | 283 -------
 .../dataflow/spark/EvaluationResult.java        |  62 --
 .../dataflow/spark/MultiDoFnFunction.java       | 115 ---
 .../dataflow/spark/ShardNameBuilder.java        | 106 ---
 .../dataflow/spark/ShardNameTemplateAware.java  |  28 -
 .../dataflow/spark/ShardNameTemplateHelper.java |  58 --
 .../dataflow/spark/SparkContextFactory.java     |  66 --
 .../dataflow/spark/SparkPipelineEvaluator.java  |  52 --
 .../dataflow/spark/SparkPipelineOptions.java    |  39 -
 .../spark/SparkPipelineOptionsFactory.java      |  27 -
 .../spark/SparkPipelineOptionsRegistrar.java    |  27 -
 .../dataflow/spark/SparkPipelineRunner.java     | 252 ------
 .../spark/SparkPipelineRunnerRegistrar.java     |  27 -
 .../dataflow/spark/SparkPipelineTranslator.java |  27 -
 .../dataflow/spark/SparkProcessContext.java     | 250 ------
 .../dataflow/spark/SparkRuntimeContext.java     | 212 -----
 .../spark/TemplatedAvroKeyOutputFormat.java     |  40 -
 .../TemplatedSequenceFileOutputFormat.java      |  40 -
 .../spark/TemplatedTextOutputFormat.java        |  40 -
 .../dataflow/spark/TransformEvaluator.java      |  24 -
 .../dataflow/spark/TransformTranslator.java     | 800 ------------------
 .../dataflow/spark/WindowingHelpers.java        |  59 --
 .../spark/aggregators/AggAccumParam.java        |  35 -
 .../spark/aggregators/NamedAggregators.java     | 202 -----
 .../SparkStreamingPipelineOptions.java          |  40 -
 .../SparkStreamingPipelineOptionsFactory.java   |  27 -
 .../SparkStreamingPipelineOptionsRegistrar.java |  28 -
 .../streaming/StreamingEvaluationContext.java   | 226 ------
 .../streaming/StreamingTransformTranslator.java | 414 ----------
 .../StreamingWindowPipelineDetector.java        | 100 ---
 .../beam/runners/spark/EvaluationResult.java    |  65 ++
 .../runners/spark/SparkPipelineOptions.java     |  42 +
 .../beam/runners/spark/SparkPipelineRunner.java | 255 ++++++
 .../spark/SparkStreamingPipelineOptions.java    |  41 +
 .../spark/aggregators/AggAccumParam.java        |  38 +
 .../spark/aggregators/NamedAggregators.java     | 205 +++++
 .../beam/runners/spark/coders/CoderHelpers.java | 189 +++++
 .../runners/spark/coders/NullWritableCoder.java |  74 ++
 .../runners/spark/coders/WritableCoder.java     | 123 +++
 .../apache/beam/runners/spark/io/ConsoleIO.java |  63 ++
 .../beam/runners/spark/io/CreateStream.java     |  69 ++
 .../apache/beam/runners/spark/io/KafkaIO.java   | 131 +++
 .../beam/runners/spark/io/hadoop/HadoopIO.java  | 203 +++++
 .../spark/io/hadoop/ShardNameBuilder.java       | 109 +++
 .../spark/io/hadoop/ShardNameTemplateAware.java |  31 +
 .../io/hadoop/ShardNameTemplateHelper.java      |  61 ++
 .../io/hadoop/TemplatedAvroKeyOutputFormat.java |  43 +
 .../TemplatedSequenceFileOutputFormat.java      |  43 +
 .../io/hadoop/TemplatedTextOutputFormat.java    |  43 +
 .../runners/spark/translation/DoFnFunction.java |  97 +++
 .../spark/translation/EvaluationContext.java    | 288 +++++++
 .../spark/translation/MultiDoFnFunction.java    | 119 +++
 .../spark/translation/SparkContextFactory.java  |  69 ++
 .../translation/SparkPipelineEvaluator.java     |  56 ++
 .../SparkPipelineOptionsFactory.java            |  31 +
 .../SparkPipelineOptionsRegistrar.java          |  31 +
 .../SparkPipelineRunnerRegistrar.java           |  31 +
 .../translation/SparkPipelineTranslator.java    |  30 +
 .../spark/translation/SparkProcessContext.java  | 262 ++++++
 .../spark/translation/SparkRuntimeContext.java  | 217 +++++
 .../spark/translation/TransformEvaluator.java   |  27 +
 .../spark/translation/TransformTranslator.java  | 808 +++++++++++++++++++
 .../spark/translation/WindowingHelpers.java     |  62 ++
 .../SparkStreamingPipelineOptionsFactory.java   |  31 +
 .../SparkStreamingPipelineOptionsRegistrar.java |  32 +
 .../streaming/StreamingEvaluationContext.java   | 229 ++++++
 .../streaming/StreamingTransformTranslator.java | 418 ++++++++++
 .../StreamingWindowPipelineDetector.java        | 104 +++
 .../runners/spark/util/BroadcastHelper.java     | 125 +++
 .../beam/runners/spark/util/ByteArray.java      |  55 ++
 ...ataflow.sdk.options.PipelineOptionsRegistrar |   4 +-
 ...dataflow.sdk.runners.PipelineRunnerRegistrar |   2 +-
 .../dataflow/hadoop/WritableCoderTest.java      |  42 -
 .../dataflow/spark/AvroPipelineTest.java        | 103 ---
 .../dataflow/spark/CombineGloballyTest.java     |  87 --
 .../dataflow/spark/CombinePerKeyTest.java       |  69 --
 .../com/cloudera/dataflow/spark/DeDupTest.java  |  55 --
 .../cloudera/dataflow/spark/DoFnOutputTest.java |  57 --
 .../cloudera/dataflow/spark/EmptyInputTest.java |  64 --
 .../spark/HadoopFileFormatPipelineTest.java     | 105 ---
 .../spark/MultiOutputWordCountTest.java         | 148 ----
 .../cloudera/dataflow/spark/NumShardsTest.java  |  89 --
 .../dataflow/spark/SerializationTest.java       | 183 -----
 .../dataflow/spark/ShardNameBuilderTest.java    |  82 --
 .../dataflow/spark/SideEffectsTest.java         |  77 --
 .../dataflow/spark/SimpleWordCountTest.java     | 117 ---
 .../spark/TestSparkPipelineOptionsFactory.java  |  34 -
 .../com/cloudera/dataflow/spark/TfIdfTest.java  |  60 --
 .../dataflow/spark/TransformTranslatorTest.java |  95 ---
 .../dataflow/spark/WindowedWordCountTest.java   |  63 --
 .../spark/streaming/FlattenStreamingTest.java   |  84 --
 .../spark/streaming/KafkaStreamingTest.java     | 133 ---
 .../streaming/SimpleStreamingWordCountTest.java |  73 --
 .../utils/DataflowAssertStreaming.java          |  39 -
 .../streaming/utils/EmbeddedKafkaCluster.java   | 314 -------
 .../apache/beam/runners/spark/DeDupTest.java    |  60 ++
 .../beam/runners/spark/EmptyInputTest.java      |  69 ++
 .../beam/runners/spark/SimpleWordCountTest.java | 115 +++
 .../apache/beam/runners/spark/TfIdfTest.java    |  64 ++
 .../runners/spark/coders/WritableCoderTest.java |  45 ++
 .../beam/runners/spark/io/AvroPipelineTest.java | 108 +++
 .../beam/runners/spark/io/NumShardsTest.java    |  96 +++
 .../io/hadoop/HadoopFileFormatPipelineTest.java | 113 +++
 .../spark/io/hadoop/ShardNameBuilderTest.java   |  85 ++
 .../spark/translation/CombineGloballyTest.java  |  94 +++
 .../spark/translation/CombinePerKeyTest.java    |  70 ++
 .../spark/translation/DoFnOutputTest.java       |  64 ++
 .../translation/MultiOutputWordCountTest.java   | 137 ++++
 .../spark/translation/SerializationTest.java    | 183 +++++
 .../spark/translation/SideEffectsTest.java      |  81 ++
 .../TestSparkPipelineOptionsFactory.java        |  38 +
 .../translation/TransformTranslatorTest.java    |  99 +++
 .../translation/WindowedWordCountTest.java      |  71 ++
 .../streaming/FlattenStreamingTest.java         |  88 ++
 .../streaming/KafkaStreamingTest.java           | 140 ++++
 .../streaming/SimpleStreamingWordCountTest.java |  77 ++
 .../utils/DataflowAssertStreaming.java          |  42 +
 .../streaming/utils/EmbeddedKafkaCluster.java   | 317 ++++++++
 135 files changed, 7362 insertions(+), 7163 deletions(-)
----------------------------------------------------------------------



[06/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
new file mode 100644
index 0000000..fd6f5da
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.io.IOException;
+
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class ShardNameTemplateHelper {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ShardNameTemplateHelper.class);
+
+  public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.fileoutputformat.prefix";
+  public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.fileoutputformat.template";
+  public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.fileoutputformat.suffix";
+
+  private ShardNameTemplateHelper() {
+  }
+
+  public static <K, V> Path getDefaultWorkFile(FileOutputFormat<K, V> format,
+      TaskAttemptContext context) throws IOException {
+    FileOutputCommitter committer =
+        (FileOutputCommitter) format.getOutputCommitter(context);
+    return new Path(committer.getWorkPath(), getOutputFile(context));
+  }
+
+  private static String getOutputFile(TaskAttemptContext context) {
+    TaskID taskId = context.getTaskAttemptID().getTaskID();
+    int partition = taskId.getId();
+
+    String filePrefix = context.getConfiguration().get(OUTPUT_FILE_PREFIX);
+    String fileTemplate = context.getConfiguration().get(OUTPUT_FILE_TEMPLATE);
+    String fileSuffix = context.getConfiguration().get(OUTPUT_FILE_SUFFIX);
+    return filePrefix + replaceShardNumber(fileTemplate, partition) + fileSuffix;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
new file mode 100644
index 0000000..4feaff6
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.avro.mapreduce.AvroKeyOutputFormat;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TemplatedAvroKeyOutputFormat<T> extends AvroKeyOutputFormat<T>
+    implements ShardNameTemplateAware {
+
+  @Override
+  public void checkOutputSpecs(JobContext job) {
+    // don't fail if the output already exists
+  }
+
+  @Override
+  protected OutputStream getAvroFileOutputStream(TaskAttemptContext context) throws IOException {
+    Path path = ShardNameTemplateHelper.getDefaultWorkFile(this, context);
+    return path.getFileSystem(context.getConfiguration()).create(path);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
new file mode 100644
index 0000000..922b906
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+
+public class TemplatedSequenceFileOutputFormat<K, V> extends SequenceFileOutputFormat<K, V>
+    implements ShardNameTemplateAware {
+
+  @Override
+  public void checkOutputSpecs(JobContext job) {
+    // don't fail if the output already exists
+  }
+
+  @Override
+  public Path getDefaultWorkFile(TaskAttemptContext context,
+      String extension) throws IOException {
+    // note that the passed-in extension is ignored since it comes from the template
+    return ShardNameTemplateHelper.getDefaultWorkFile(this, context);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
new file mode 100644
index 0000000..1e53dce
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.io.hadoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+
+public class TemplatedTextOutputFormat<K, V> extends TextOutputFormat<K, V>
+    implements ShardNameTemplateAware {
+
+  @Override
+  public void checkOutputSpecs(JobContext job) {
+    // don't fail if the output already exists
+  }
+
+  @Override
+  public Path getDefaultWorkFile(TaskAttemptContext context,
+      String extension) throws IOException {
+    // note that the passed-in extension is ignored since it comes from the template
+    return ShardNameTemplateHelper.getDefaultWorkFile(this, context);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
new file mode 100644
index 0000000..17edba3
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptions.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+
+/**
+ * Options used to configure Spark streaming.
+ */
+public interface SparkStreamingPipelineOptions extends SparkPipelineOptions {
+  @Description("Timeout to wait (in msec) for the streaming execution so stop, -1 runs until " +
+          "execution is stopped")
+  @Default.Long(-1)
+  Long getTimeout();
+
+  void setTimeout(Long batchInterval);
+
+  @Override
+  @Default.Boolean(true)
+  boolean isStreaming();
+
+  @Override
+  @Default.String("spark streaming dataflow pipeline job")
+  String getAppName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
new file mode 100644
index 0000000..822feb4
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+
+public final class SparkStreamingPipelineOptionsFactory {
+
+  private SparkStreamingPipelineOptionsFactory() {
+  }
+
+  public static SparkStreamingPipelineOptions create() {
+    return PipelineOptionsFactory.as(SparkStreamingPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
new file mode 100644
index 0000000..2c5414d
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+
+public class SparkStreamingPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
+
+  @Override
+  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkStreamingPipelineOptions
+            .class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
new file mode 100644
index 0000000..9d1d786
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingEvaluationContext.java
@@ -0,0 +1,226 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+
+import org.apache.beam.runners.spark.EvaluationContext;
+import org.apache.beam.runners.spark.SparkRuntimeContext;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaDStreamLike;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+
+/**
+ * Streaming evaluation context helps to handle streaming.
+ */
+public class StreamingEvaluationContext extends EvaluationContext {
+
+  private final JavaStreamingContext jssc;
+  private final long timeout;
+  private final Map<PValue, DStreamHolder<?>> pstreams = new LinkedHashMap<>();
+  private final Set<DStreamHolder<?>> leafStreams = new LinkedHashSet<>();
+
+  public StreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
+      JavaStreamingContext jssc, long timeout) {
+    super(jsc, pipeline);
+    this.jssc = jssc;
+    this.timeout = timeout;
+  }
+
+  /**
+   * DStream holder Can also crate a DStream from a supplied queue of values, but mainly for
+   * testing.
+   */
+  private class DStreamHolder<T> {
+
+    private Iterable<Iterable<T>> values;
+    private Coder<T> coder;
+    private JavaDStream<WindowedValue<T>> dStream;
+
+    DStreamHolder(Iterable<Iterable<T>> values, Coder<T> coder) {
+      this.values = values;
+      this.coder = coder;
+    }
+
+    DStreamHolder(JavaDStream<WindowedValue<T>> dStream) {
+      this.dStream = dStream;
+    }
+
+    @SuppressWarnings("unchecked")
+    JavaDStream<WindowedValue<T>> getDStream() {
+      if (dStream == null) {
+        // create the DStream from values
+        Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
+        for (Iterable<T> v : values) {
+          setOutputRDDFromValues(currentTransform.getTransform(), v, coder);
+          rddQueue.offer((JavaRDD<WindowedValue<T>>) getOutputRDD(currentTransform.getTransform()));
+        }
+        // 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);
+      }
+      return dStream;
+    }
+  }
+
+  <T> void setDStreamFromQueue(
+      PTransform<?, ?> transform, Iterable<Iterable<T>> values, Coder<T> coder) {
+    pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder));
+  }
+
+  <T> void setStream(PTransform<?, ?> transform, JavaDStream<WindowedValue<T>> dStream) {
+    PValue pvalue = (PValue) getOutput(transform);
+    DStreamHolder<T> dStreamHolder = new DStreamHolder<>(dStream);
+    pstreams.put(pvalue, dStreamHolder);
+    leafStreams.add(dStreamHolder);
+  }
+
+  boolean hasStream(PTransform<?, ?> transform) {
+    PValue pvalue = (PValue) getInput(transform);
+    return pstreams.containsKey(pvalue);
+  }
+
+  JavaDStreamLike<?, ?, ?> getStream(PTransform<?, ?> transform) {
+    return getStream((PValue) getInput(transform));
+  }
+
+  JavaDStreamLike<?, ?, ?> getStream(PValue pvalue) {
+    DStreamHolder<?> dStreamHolder = pstreams.get(pvalue);
+    JavaDStreamLike<?, ?, ?> dStream = dStreamHolder.getDStream();
+    leafStreams.remove(dStreamHolder);
+    return dStream;
+  }
+
+  // used to set the RDD from the DStream in the RDDHolder for transformation
+  <T> void setInputRDD(
+      PTransform<? extends PInput, ?> transform, JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    setRDD((PValue) getInput(transform), rdd);
+  }
+
+  // used to get the RDD transformation output and use it as the DStream transformation output
+  JavaRDDLike<?, ?> getOutputRDD(PTransform<?, ?> transform) {
+    return getRDD((PValue) getOutput(transform));
+  }
+
+  public JavaStreamingContext getStreamingContext() {
+    return jssc;
+  }
+
+  @Override
+  protected void computeOutputs() {
+    for (DStreamHolder<?> streamHolder : leafStreams) {
+      computeOutput(streamHolder);
+    }
+  }
+
+  private static <T> void computeOutput(DStreamHolder<T> streamHolder) {
+    streamHolder.getDStream().foreachRDD(new Function<JavaRDD<WindowedValue<T>>, Void>() {
+      @Override
+      public Void call(JavaRDD<WindowedValue<T>> rdd) throws Exception {
+        rdd.rdd().cache();
+        rdd.count();
+        return null;
+      }
+    }); // force a DStream action
+  }
+
+  @Override
+  public void close() {
+    if (timeout > 0) {
+      jssc.awaitTerminationOrTimeout(timeout);
+    } else {
+      jssc.awaitTermination();
+    }
+    //TODO: stop gracefully ?
+    jssc.stop(false, false);
+    state = State.DONE;
+    super.close();
+  }
+
+  private State state = State.RUNNING;
+
+  @Override
+  public State getState() {
+    return state;
+  }
+
+  //---------------- override in order to expose in package
+  @Override
+  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
+    return super.getInput(transform);
+  }
+  @Override
+  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
+    return super.getOutput(transform);
+  }
+
+  @Override
+  protected JavaSparkContext getSparkContext() {
+    return super.getSparkContext();
+  }
+
+  @Override
+  protected SparkRuntimeContext getRuntimeContext() {
+    return super.getRuntimeContext();
+  }
+
+  @Override
+  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
+    super.setCurrentTransform(transform);
+  }
+
+  @Override
+  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
+    return super.getCurrentTransform();
+  }
+
+  @Override
+  protected <T> void setOutputRDD(PTransform<?, ?> transform,
+      JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    super.setOutputRDD(transform, rdd);
+  }
+
+  @Override
+  protected <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
+      Coder<T> coder) {
+    super.setOutputRDDFromValues(transform, values, coder);
+  }
+
+  @Override
+  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
+    return super.hasOutputRDD(transform);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
new file mode 100644
index 0000000..c78c7fa
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingTransformTranslator.java
@@ -0,0 +1,415 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.streaming;
+
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.api.client.util.Lists;
+import com.google.api.client.util.Maps;
+import com.google.api.client.util.Sets;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+import com.google.common.reflect.TypeToken;
+import kafka.serializer.Decoder;
+
+import org.apache.beam.runners.spark.DoFnFunction;
+import org.apache.beam.runners.spark.EvaluationContext;
+import org.apache.beam.runners.spark.SparkPipelineTranslator;
+import org.apache.beam.runners.spark.TransformEvaluator;
+import org.apache.beam.runners.spark.TransformTranslator;
+import org.apache.beam.runners.spark.WindowingHelpers;
+import org.apache.beam.runners.spark.io.ConsoleIO;
+import org.apache.beam.runners.spark.io.CreateStream;
+import org.apache.beam.runners.spark.io.KafkaIO;
+import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaDStreamLike;
+import org.apache.spark.streaming.api.java.JavaPairInputDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.kafka.KafkaUtils;
+
+import scala.Tuple2;
+
+
+/**
+ * Supports translation between a DataFlow transform, and Spark's operations on DStreams.
+ */
+public final class StreamingTransformTranslator {
+
+  private StreamingTransformTranslator() {
+  }
+
+  private static <T> TransformEvaluator<ConsoleIO.Write.Unbound<T>> print() {
+    return new TransformEvaluator<ConsoleIO.Write.Unbound<T>>() {
+      @Override
+      public void evaluate(ConsoleIO.Write.Unbound<T> transform, EvaluationContext context) {
+        @SuppressWarnings("unchecked")
+        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
+            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
+            ((StreamingEvaluationContext) context).getStream(transform);
+        dstream.map(WindowingHelpers.<T>unwindowFunction()).print(transform.getNum());
+      }
+    };
+  }
+
+  private static <K, V> TransformEvaluator<KafkaIO.Read.Unbound<K, V>> kafka() {
+    return new TransformEvaluator<KafkaIO.Read.Unbound<K, V>>() {
+      @Override
+      public void evaluate(KafkaIO.Read.Unbound<K, V> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        JavaStreamingContext jssc = sec.getStreamingContext();
+        Class<K> keyClazz = transform.getKeyClass();
+        Class<V> valueClazz = transform.getValueClass();
+        Class<? extends Decoder<K>> keyDecoderClazz = transform.getKeyDecoderClass();
+        Class<? extends Decoder<V>> valueDecoderClazz = transform.getValueDecoderClass();
+        Map<String, String> kafkaParams = transform.getKafkaParams();
+        Set<String> topics = transform.getTopics();
+        JavaPairInputDStream<K, V> inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz,
+                valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics);
+        JavaDStream<WindowedValue<KV<K, V>>> inputStream =
+            inputPairStream.map(new Function<Tuple2<K, V>, KV<K, V>>() {
+          @Override
+          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
+            return KV.of(t2._1(), t2._2());
+          }
+        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
+        sec.setStream(transform, inputStream);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Create.Values<T>> create() {
+    return new TransformEvaluator<Create.Values<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        Iterable<T> elems = transform.getElements();
+        Coder<T> coder = sec.getOutput(transform).getCoder();
+        if (coder != VoidCoder.of()) {
+          // actual create
+          sec.setOutputRDDFromValues(transform, elems, coder);
+        } else {
+          // fake create as an input
+          // creates a stream with a single batch containing a single null element
+          // to invoke following transformations once
+          // to support DataflowAssert
+          sec.setDStreamFromQueue(transform,
+              Collections.<Iterable<Void>>singletonList(Collections.singletonList((Void) null)),
+              (Coder<Void>) coder);
+        }
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<CreateStream.QueuedValues<T>> createFromQueue() {
+    return new TransformEvaluator<CreateStream.QueuedValues<T>>() {
+      @Override
+      public void evaluate(CreateStream.QueuedValues<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        Iterable<Iterable<T>> values = transform.getQueuedValues();
+        Coder<T> coder = sec.getOutput(transform).getCoder();
+        sec.setDStreamFromQueue(transform, values, coder);
+      }
+    };
+  }
+
+  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
+    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        PCollectionList<T> pcs = sec.getInput(transform);
+        JavaDStream<WindowedValue<T>> first =
+            (JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(0));
+        List<JavaDStream<WindowedValue<T>>> rest = Lists.newArrayListWithCapacity(pcs.size() - 1);
+        for (int i = 1; i < pcs.size(); i++) {
+          rest.add((JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(i)));
+        }
+        JavaDStream<WindowedValue<T>> dstream = sec.getStreamingContext().union(first, rest);
+        sec.setStream(transform, dstream);
+      }
+    };
+  }
+
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> rddTransform(
+      final SparkPipelineTranslator rddTranslator) {
+    return new TransformEvaluator<PT>() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator<PT> rddEvaluator =
+            rddTranslator.translate((Class<PT>) transform.getClass());
+
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        if (sec.hasStream(transform)) {
+          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
+              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
+              sec.getStream(transform);
+
+          sec.setStream(transform, dStream
+              .transform(new RDDTransform<>(sec, rddEvaluator, transform)));
+        } else {
+          // if the transformation requires direct access to RDD (not in stream)
+          // this is used for "fake" transformations like with DataflowAssert
+          rddEvaluator.evaluate(transform, context);
+        }
+      }
+    };
+  }
+
+  /**
+   * RDD transform function If the transformation function doesn't have an input, create a fake one
+   * as an empty RDD.
+   *
+   * @param <PT> PTransform type
+   */
+  private static final class RDDTransform<PT extends PTransform<?, ?>>
+      implements Function<JavaRDD<WindowedValue<Object>>, JavaRDD<WindowedValue<Object>>> {
+
+    private final StreamingEvaluationContext context;
+    private final AppliedPTransform<?, ?, ?> appliedPTransform;
+    private final TransformEvaluator<PT> rddEvaluator;
+    private final PT transform;
+
+
+    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator<PT> rddEvaluator,
+        PT transform) {
+      this.context = context;
+      this.appliedPTransform = context.getCurrentTransform();
+      this.rddEvaluator = rddEvaluator;
+      this.transform = transform;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public JavaRDD<WindowedValue<Object>>
+        call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
+      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
+      context.setCurrentTransform(appliedPTransform);
+      context.setInputRDD(transform, rdd);
+      rddEvaluator.evaluate(transform, context);
+      if (!context.hasOutputRDD(transform)) {
+        // fake RDD as output
+        context.setOutputRDD(transform,
+            context.getSparkContext().<WindowedValue<Object>>emptyRDD());
+      }
+      JavaRDD<WindowedValue<Object>> outRDD =
+          (JavaRDD<WindowedValue<Object>>) context.getOutputRDD(transform);
+      context.setCurrentTransform(existingAPT);
+      return outRDD;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> foreachRDD(
+      final SparkPipelineTranslator rddTranslator) {
+    return new TransformEvaluator<PT>() {
+      @Override
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator<PT> rddEvaluator =
+            rddTranslator.translate((Class<PT>) transform.getClass());
+
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        if (sec.hasStream(transform)) {
+          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
+              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
+              sec.getStream(transform);
+
+          dStream.foreachRDD(new RDDOutputOperator<>(sec, rddEvaluator, transform));
+        } else {
+          rddEvaluator.evaluate(transform, context);
+        }
+      }
+    };
+  }
+
+  /**
+   * RDD output function.
+   *
+   * @param <PT> PTransform type
+   */
+  private static final class RDDOutputOperator<PT extends PTransform<?, ?>>
+      implements Function<JavaRDD<WindowedValue<Object>>, Void> {
+
+    private final StreamingEvaluationContext context;
+    private final AppliedPTransform<?, ?, ?> appliedPTransform;
+    private final TransformEvaluator<PT> rddEvaluator;
+    private final PT transform;
+
+
+    private RDDOutputOperator(StreamingEvaluationContext context,
+        TransformEvaluator<PT> rddEvaluator, PT transform) {
+      this.context = context;
+      this.appliedPTransform = context.getCurrentTransform();
+      this.rddEvaluator = rddEvaluator;
+      this.transform = transform;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Void call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
+      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
+      context.setCurrentTransform(appliedPTransform);
+      context.setInputRDD(transform, rdd);
+      rddEvaluator.evaluate(transform, context);
+      context.setCurrentTransform(existingAPT);
+      return null;
+    }
+  }
+
+  private static final TransformTranslator.FieldGetter WINDOW_FG =
+      new TransformTranslator.FieldGetter(Window.Bound.class);
+
+  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
+    return new TransformEvaluator<Window.Bound<T>>() {
+      @Override
+      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
+        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
+        //--- first we apply windowing to the stream
+        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
+        @SuppressWarnings("unchecked")
+        JavaDStream<WindowedValue<T>> dStream =
+            (JavaDStream<WindowedValue<T>>) sec.getStream(transform);
+        if (windowFn instanceof FixedWindows) {
+          Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize()
+              .getMillis());
+          sec.setStream(transform, dStream.window(windowDuration));
+        } else if (windowFn instanceof SlidingWindows) {
+          Duration windowDuration = Durations.milliseconds(((SlidingWindows) windowFn).getSize()
+              .getMillis());
+          Duration slideDuration = Durations.milliseconds(((SlidingWindows) windowFn).getPeriod()
+              .getMillis());
+          sec.setStream(transform, dStream.window(windowDuration, slideDuration));
+        }
+        //--- then we apply windowing to the elements
+        DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
+        DoFnFunction<T, T> dofn = new DoFnFunction<>(addWindowsDoFn,
+            ((StreamingEvaluationContext)context).getRuntimeContext(), null);
+        @SuppressWarnings("unchecked")
+        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
+            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
+            sec.getStream(transform);
+        sec.setStream(transform, dstream.mapPartitions(dofn));
+      }
+    };
+  }
+
+  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
+      .newHashMap();
+
+  static {
+    EVALUATORS.put(ConsoleIO.Write.Unbound.class, print());
+    EVALUATORS.put(CreateStream.QueuedValues.class, createFromQueue());
+    EVALUATORS.put(Create.Values.class, create());
+    EVALUATORS.put(KafkaIO.Read.Unbound.class, kafka());
+    EVALUATORS.put(Window.Bound.class, window());
+    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
+  }
+
+  private static final Set<Class<? extends PTransform>> UNSUPPORTED_EVALUATORS = Sets
+      .newHashSet();
+
+  static {
+    //TODO - add support for the following
+    UNSUPPORTED_EVALUATORS.add(TextIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(TextIO.Write.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(AvroIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(AvroIO.Write.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(HadoopIO.Read.Bound.class);
+    UNSUPPORTED_EVALUATORS.add(HadoopIO.Write.Bound.class);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
+      getTransformEvaluator(Class<PT> clazz, SparkPipelineTranslator rddTranslator) {
+    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
+    if (transform == null) {
+      if (UNSUPPORTED_EVALUATORS.contains(clazz)) {
+        throw new UnsupportedOperationException("Dataflow transformation " + clazz
+          .getCanonicalName()
+          + " is currently unsupported by the Spark streaming pipeline");
+      }
+      // DStream transformations will transform an RDD into another RDD
+      // Actions will create output
+      // In Dataflow it depends on the PTransform's Input and Output class
+      Class<?> pTOutputClazz = getPTransformOutputClazz(clazz);
+      if (PDone.class.equals(pTOutputClazz)) {
+        return foreachRDD(rddTranslator);
+      } else {
+        return rddTransform(rddTranslator);
+      }
+    }
+    return transform;
+  }
+
+  private static <PT extends PTransform<?, ?>> Class<?> getPTransformOutputClazz(Class<PT> clazz) {
+    Type[] types = ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments();
+    return TypeToken.of(clazz).resolveType(types[1]).getRawType();
+  }
+
+  /**
+   * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator.
+   * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation
+   */
+  public static class Translator implements SparkPipelineTranslator {
+
+    private final SparkPipelineTranslator rddTranslator;
+
+    public Translator(SparkPipelineTranslator rddTranslator) {
+      this.rddTranslator = rddTranslator;
+    }
+
+    @Override
+    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
+      // streaming includes rdd transformations as well
+      return EVALUATORS.containsKey(clazz) || rddTranslator.hasTranslation(clazz);
+    }
+
+    @Override
+    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
+      return getTransformEvaluator(clazz, rddTranslator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
new file mode 100644
index 0000000..6844011
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/streaming/StreamingWindowPipelineDetector.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.streaming;
+
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.SparkPipelineTranslator;
+import org.apache.beam.runners.spark.TransformTranslator;
+
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Durations;
+
+
+/**
+ * Pipeline {@link SparkPipelineRunner.Evaluator} to detect windowing.
+ */
+public final class StreamingWindowPipelineDetector extends SparkPipelineRunner.Evaluator {
+
+  // Currently, Spark streaming recommends batches no smaller then 500 msec
+  private static final Duration SPARK_MIN_WINDOW = Durations.milliseconds(500);
+
+  private boolean windowing;
+  private Duration batchDuration;
+
+  public StreamingWindowPipelineDetector(SparkPipelineTranslator translator) {
+    super(translator);
+  }
+
+  private static final TransformTranslator.FieldGetter WINDOW_FG =
+      new TransformTranslator.FieldGetter(Window.Bound.class);
+
+  // Use the smallest window (fixed or sliding) as Spark streaming's batch duration
+  @Override
+  protected <PT extends PTransform<? super PInput, POutput>> void
+      doVisitTransform(TransformTreeNode node) {
+    @SuppressWarnings("unchecked")
+    PT transform = (PT) node.getTransform();
+    @SuppressWarnings("unchecked")
+    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
+    if (transformClass.isAssignableFrom(Window.Bound.class)) {
+      WindowFn<?, ?> windowFn = WINDOW_FG.get("windowFn", transform);
+      if (windowFn instanceof FixedWindows) {
+        setBatchDuration(((FixedWindows) windowFn).getSize());
+      } else if (windowFn instanceof SlidingWindows) {
+        if (((SlidingWindows) windowFn).getOffset().getMillis() > 0) {
+          throw new UnsupportedOperationException("Spark does not support window offsets");
+        }
+        // Sliding window size might as well set the batch duration. Applying the transformation
+        // will add the "slide"
+        setBatchDuration(((SlidingWindows) windowFn).getSize());
+      } else if (!(windowFn instanceof GlobalWindows)) {
+        throw new IllegalStateException("Windowing function not supported: " + windowFn);
+      }
+    }
+  }
+
+  private void setBatchDuration(org.joda.time.Duration duration) {
+    Long durationMillis = duration.getMillis();
+    // validate window size
+    if (durationMillis < SPARK_MIN_WINDOW.milliseconds()) {
+      throw new IllegalArgumentException("Windowing of size " + durationMillis +
+          "msec is not supported!");
+    }
+    // choose the smallest duration to be Spark's batch duration, larger ones will be handled
+    // as window functions  over the batched-stream
+    if (!windowing || this.batchDuration.milliseconds() > durationMillis) {
+      this.batchDuration = Durations.milliseconds(durationMillis);
+    }
+    windowing = true;
+  }
+
+  public boolean isWindowing() {
+    return windowing;
+  }
+
+  public Duration getBatchDuration() {
+    return batchDuration;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
new file mode 100644
index 0000000..af831c6
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.Serializable;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BroadcastHelper<T> implements Serializable {
+
+  /**
+   * If the property {@code dataflow.spark.directBroadcast} is set to
+   * {@code true} then Spark serialization (Kryo) will be used to broadcast values
+   * in View objects. By default this property is not set, and values are coded using
+   * the appropriate {@link Coder}.
+   */
+  public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast";
+
+  private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class);
+
+  public static <T> BroadcastHelper<T> create(T value, Coder<T> coder) {
+    if (Boolean.parseBoolean(System.getProperty(DIRECT_BROADCAST, "false"))) {
+      return new DirectBroadcastHelper<>(value);
+    }
+    return new CodedBroadcastHelper<>(value, coder);
+  }
+
+  public abstract T getValue();
+
+  public abstract void broadcast(JavaSparkContext jsc);
+
+  /**
+   * A {@link BroadcastHelper} that relies on the underlying
+   * Spark serialization (Kryo) to broadcast values. This is appropriate when
+   * broadcasting very large values, since no copy of the object is made.
+   * @param <T>
+   */
+  static class DirectBroadcastHelper<T> extends BroadcastHelper<T> {
+    private Broadcast<T> bcast;
+    private transient T value;
+
+    DirectBroadcastHelper(T value) {
+      this.value = value;
+    }
+
+    @Override
+    public synchronized T getValue() {
+      if (value == null) {
+        value = bcast.getValue();
+      }
+      return value;
+    }
+
+    @Override
+    public void broadcast(JavaSparkContext jsc) {
+      this.bcast = jsc.broadcast(value);
+    }
+  }
+
+  /**
+   * A {@link BroadcastHelper} that uses a
+   * {@link Coder} to encode values as byte arrays
+   * before broadcasting.
+   * @param <T>
+   */
+  static class CodedBroadcastHelper<T> extends BroadcastHelper<T> {
+    private Broadcast<byte[]> bcast;
+    private final Coder<T> coder;
+    private transient T value;
+
+    CodedBroadcastHelper(T value, Coder<T> coder) {
+      this.value = value;
+      this.coder = coder;
+    }
+
+    @Override
+    public synchronized T getValue() {
+      if (value == null) {
+        value = deserialize();
+      }
+      return value;
+    }
+
+    @Override
+    public void broadcast(JavaSparkContext jsc) {
+      this.bcast = jsc.broadcast(CoderHelpers.toByteArray(value, coder));
+    }
+
+    private T deserialize() {
+      T val;
+      try {
+        val = coder.decode(new ByteArrayInputStream(bcast.value()),
+            new Coder.Context(true));
+      } catch (IOException ioe) {
+        // this should not ever happen, log it if it does.
+        LOG.warn(ioe.getMessage());
+        val = null;
+      }
+      return val;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
new file mode 100644
index 0000000..7679b9c
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark.util;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+import com.google.common.primitives.UnsignedBytes;
+
+public class ByteArray implements Serializable, Comparable<ByteArray> {
+
+  private final byte[] value;
+
+  public ByteArray(byte[] value) {
+    this.value = value;
+  }
+
+  public byte[] getValue() {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ByteArray byteArray = (ByteArray) o;
+    return Arrays.equals(value, byteArray.value);
+  }
+
+  @Override
+  public int hashCode() {
+    return value != null ? Arrays.hashCode(value) : 0;
+  }
+
+  @Override
+  public int compareTo(ByteArray other) {
+    return UnsignedBytes.lexicographicalComparator().compare(value, other.value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
index 5733a86..98387a6 100644
--- a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
+++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar
@@ -13,5 +13,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-com.cloudera.dataflow.spark.SparkPipelineOptionsRegistrar
-com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptionsRegistrar
\ No newline at end of file
+org.apache.beam.runners.spark.SparkPipelineOptionsRegistrar
+org.apache.beam.runners.spark.streaming.SparkStreamingPipelineOptionsRegistrar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
index 26e0b3a..972b1a3 100644
--- a/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
+++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-com.cloudera.dataflow.spark.SparkPipelineRunnerRegistrar
\ No newline at end of file
+org.apache.beam.runners.spark.SparkPipelineRunnerRegistrar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/hadoop/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/hadoop/WritableCoderTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/hadoop/WritableCoderTest.java
deleted file mode 100644
index 29a73b6..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/hadoop/WritableCoderTest.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.hadoop;
-
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.junit.Test;
-
-/**
- * Tests for WritableCoder.
- */
-public class WritableCoderTest {
-
-  @Test
-  public void testIntWritableEncoding() throws Exception {
-    IntWritable value = new IntWritable(42);
-    WritableCoder<IntWritable> coder = WritableCoder.of(IntWritable.class);
-
-    CoderProperties.coderDecodeEncodeEqual(coder, value);
-  }
-
-  @Test
-  public void testNullWritableEncoding() throws Exception {
-    WritableCoder<NullWritable> coder = WritableCoder.of(NullWritable.class);
-
-    CoderProperties.coderDecodeEncodeEqual(coder, NullWritable.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java
deleted file mode 100644
index ea4cc38..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Lists;
-import com.google.common.io.Resources;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.List;
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import static org.junit.Assert.assertEquals;
-
-public class AvroPipelineTest {
-
-  private File inputFile;
-  private File outputDir;
-
-  @Rule
-  public final TemporaryFolder tmpDir = new TemporaryFolder();
-
-  @Before
-  public void setUp() throws IOException {
-    inputFile = tmpDir.newFile("test.avro");
-    outputDir = tmpDir.newFolder("out");
-    outputDir.delete();
-  }
-
-  @Test
-  public void testGeneric() throws Exception {
-    Schema schema = new Schema.Parser().parse(Resources.getResource("person.avsc").openStream());
-    GenericRecord savedRecord = new GenericData.Record(schema);
-    savedRecord.put("name", "John Doe");
-    savedRecord.put("age", 42);
-    savedRecord.put("siblingnames", Lists.newArrayList("Jimmy", "Jane"));
-    populateGenericFile(Lists.newArrayList(savedRecord), schema);
-
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    PCollection<GenericRecord> input = p.apply(
-        AvroIO.Read.from(inputFile.getAbsolutePath()).withSchema(schema));
-    input.apply(AvroIO.Write.to(outputDir.getAbsolutePath()).withSchema(schema));
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-
-    List<GenericRecord> records = readGenericFile();
-    assertEquals(Lists.newArrayList(savedRecord), records);
-  }
-
-  private void populateGenericFile(List<GenericRecord> genericRecords, Schema schema) throws IOException {
-    FileOutputStream outputStream = new FileOutputStream(this.inputFile);
-    GenericDatumWriter<GenericRecord> genericDatumWriter = new GenericDatumWriter<>(schema);
-
-    try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(genericDatumWriter)) {
-      dataFileWriter.create(schema, outputStream);
-      for (GenericRecord record : genericRecords) {
-        dataFileWriter.append(record);
-      }
-    }
-    outputStream.close();
-  }
-
-  private List<GenericRecord> readGenericFile() throws IOException {
-    List<GenericRecord> records = Lists.newArrayList();
-    GenericDatumReader<GenericRecord> genericDatumReader = new GenericDatumReader<>();
-    try (DataFileReader<GenericRecord> dataFileReader =
-             new DataFileReader<>(new File(outputDir + "-00000-of-00001"), genericDatumReader)) {
-      for (GenericRecord record : dataFileReader) {
-        records.add(record);
-      }
-    }
-    return records;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java
deleted file mode 100644
index 667e949..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Iterables;
-import java.util.Arrays;
-import java.util.List;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class CombineGloballyTest {
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    Pipeline p = Pipeline.create(options);
-    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-    PCollection<String> output = inputWords.apply(Combine.globally(new WordMerger()));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    assertEquals("hi there,hi,hi sue bob,hi sue,,bob hi", Iterables.getOnlyElement(res.get(output)));
-    res.close();
-  }
-
-  public static class WordMerger extends Combine.CombineFn<String, StringBuilder, String> {
-
-    @Override
-    public StringBuilder createAccumulator() {
-      // return null to differentiate from an empty string
-      return null;
-    }
-
-    @Override
-    public StringBuilder addInput(StringBuilder accumulator, String input) {
-      return combine(accumulator, input);
-    }
-
-    @Override
-    public StringBuilder mergeAccumulators(Iterable<StringBuilder> accumulators) {
-      StringBuilder sb = new StringBuilder();
-      for (StringBuilder accum : accumulators) {
-        if (accum != null) {
-          sb.append(accum);
-        }
-      }
-      return sb;
-    }
-
-    @Override
-    public String extractOutput(StringBuilder accumulator) {
-      return accumulator.toString();
-    }
-
-    private static StringBuilder combine(StringBuilder accum, String datum) {
-      if (accum == null) {
-        return new StringBuilder(datum);
-      } else {
-        accum.append(",").append(datum);
-        return accum;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java
deleted file mode 100644
index f9d5b46..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class CombinePerKeyTest {
-
-    private static final List<String> WORDS =
-        ImmutableList.of("the", "quick", "brown", "fox", "jumped", "over", "the", "lazy", "dog");
-    @Test
-    public void testRun() {
-        Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-        PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-        PCollection<KV<String, Long>> cnts = inputWords.apply(new SumPerKey<String>());
-        EvaluationResult res = SparkPipelineRunner.create().run(p);
-        Map<String, Long> actualCnts = new HashMap<>();
-        for (KV<String, Long> kv : res.get(cnts)) {
-            actualCnts.put(kv.getKey(), kv.getValue());
-        }
-        res.close();
-        Assert.assertEquals(8, actualCnts.size());
-        Assert.assertEquals(Long.valueOf(2L), actualCnts.get("the"));
-    }
-
-    private static class SumPerKey<T> extends PTransform<PCollection<T>, PCollection<KV<T, Long>>> {
-      @Override
-      public PCollection<KV<T, Long>> apply(PCollection<T> pcol) {
-          PCollection<KV<T, Long>> withLongs = pcol.apply(ParDo.of(new DoFn<T, KV<T, Long>>() {
-              @Override
-              public void processElement(ProcessContext processContext) throws Exception {
-                  processContext.output(KV.of(processContext.element(), 1L));
-              }
-          })).setCoder(KvCoder.of(pcol.getCoder(), VarLongCoder.of()));
-          return withLongs.apply(Sum.<T>longsPerKey());
-      }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java
deleted file mode 100644
index 7495aeb..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import org.junit.Test;
-
-/**
- * A test based on {@code DeDupExample} from the SDK.
- */
-public class DeDupTest {
-
-  private static final String[] LINES_ARRAY = {
-      "hi there", "hello", "hi there",
-      "hi", "hello"};
-  private static final List<String> LINES = Arrays.asList(LINES_ARRAY);
-  private static final Set<String> EXPECTED_SET =
-      ImmutableSet.of("hi there", "hi", "hello");
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    PCollection<String> input = p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of());
-    PCollection<String> output = input.apply(RemoveDuplicates.<String>create());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.java
deleted file mode 100644
index 2b0947f..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import java.io.Serializable;
-import org.junit.Test;
-
-public class DoFnOutputTest implements Serializable {
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    PCollection<String> strings = pipeline.apply(Create.of("a"));
-    // Test that values written from startBundle() and finishBundle() are written to
-    // the output
-    PCollection<String> output = strings.apply(ParDo.of(new DoFn<String, String>() {
-      @Override
-      public void startBundle(Context c) throws Exception {
-        c.output("start");
-      }
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(c.element());
-      }
-      @Override
-      public void finishBundle(Context c) throws Exception {
-        c.output("finish");
-      }
-    }));
-
-    DataflowAssert.that(output).containsInAnyOrder("start", "a", "finish");
-
-    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
-    res.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java
deleted file mode 100644
index 6c89ca1..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Iterables;
-import java.util.Collections;
-import java.util.List;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class EmptyInputTest {
-
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    Pipeline p = Pipeline.create(options);
-    List<String> empty = Collections.emptyList();
-    PCollection<String> inputWords = p.apply(Create.of(empty)).setCoder(StringUtf8Coder.of());
-    PCollection<String> output = inputWords.apply(Combine.globally(new ConcatWords()));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    assertEquals("", Iterables.getOnlyElement(res.get(output)));
-    res.close();
-  }
-
-  public static class ConcatWords implements SerializableFunction<Iterable<String>, String> {
-    @Override
-    public String apply(Iterable<String> input) {
-      StringBuilder all = new StringBuilder();
-      for (String item : input) {
-        if (!item.isEmpty()) {
-          if (all.length() == 0) {
-            all.append(item);
-          } else {
-            all.append(",");
-            all.append(item);
-          }
-        }
-      }
-      return all.toString();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java
deleted file mode 100644
index 579ada5..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.cloudera.dataflow.hadoop.HadoopIO;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import java.io.File;
-import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import static org.junit.Assert.assertEquals;
-
-public class HadoopFileFormatPipelineTest {
-
-  private File inputFile;
-  private File outputFile;
-
-  @Rule
-  public final TemporaryFolder tmpDir = new TemporaryFolder();
-
-  @Before
-  public void setUp() throws IOException {
-    inputFile = tmpDir.newFile("test.seq");
-    outputFile = tmpDir.newFolder("out");
-    outputFile.delete();
-  }
-
-  @Test
-  public void testSequenceFile() throws Exception {
-    populateFile();
-
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    @SuppressWarnings("unchecked")
-    Class<? extends FileInputFormat<IntWritable, Text>> inputFormatClass =
-        (Class<? extends FileInputFormat<IntWritable, Text>>) (Class<?>) SequenceFileInputFormat.class;
-    HadoopIO.Read.Bound<IntWritable,Text> read =
-        HadoopIO.Read.from(inputFile.getAbsolutePath(), inputFormatClass, IntWritable.class, Text.class);
-    PCollection<KV<IntWritable, Text>> input = p.apply(read);
-    @SuppressWarnings("unchecked")
-    Class<? extends FileOutputFormat<IntWritable, Text>> outputFormatClass =
-        (Class<? extends FileOutputFormat<IntWritable, Text>>) (Class<?>) TemplatedSequenceFileOutputFormat.class;
-    @SuppressWarnings("unchecked")
-    HadoopIO.Write.Bound<IntWritable,Text> write = HadoopIO.Write.to(outputFile.getAbsolutePath(),
-        outputFormatClass, IntWritable.class, Text.class);
-    input.apply(write.withoutSharding());
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-
-    IntWritable key = new IntWritable();
-    Text value = new Text();
-    try (Reader reader = new Reader(new Configuration(), Reader.file(new Path(outputFile.toURI())))) {
-      int i = 0;
-      while (reader.next(key, value)) {
-        assertEquals(i, key.get());
-        assertEquals("value-" + i, value.toString());
-        i++;
-      }
-    }
-  }
-
-  private void populateFile() throws IOException {
-    IntWritable key = new IntWritable();
-    Text value = new Text();
-    try (Writer writer = SequenceFile.createWriter(
-        new Configuration(),
-        Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class),
-        Writer.file(new Path(this.inputFile.toURI())))) {
-      for (int i = 0; i < 5; i++) {
-        key.set(i);
-        value.set("value-" + i);
-        writer.append(key, value);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java
deleted file mode 100644
index 2df8493..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.ApproximateUnique;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-import com.google.common.collect.Iterables;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class MultiOutputWordCountTest {
-
-  private static final TupleTag<String> upper = new TupleTag<>();
-  private static final TupleTag<String> lower = new TupleTag<>();
-  private static final TupleTag<KV<String, Long>> lowerCnts = new TupleTag<>();
-  private static final TupleTag<KV<String, Long>> upperCnts = new TupleTag<>();
-
-  @Test
-  public void testRun() throws Exception {
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    PCollection<String> regex = p.apply(Create.of("[^a-zA-Z']+"));
-    PCollection<String> w1 = p.apply(Create.of("Here are some words to count", "and some others"));
-    PCollection<String> w2 = p.apply(Create.of("Here are some more words", "and even more words"));
-    PCollectionList<String> list = PCollectionList.of(w1).and(w2);
-
-    PCollection<String> union = list.apply(Flatten.<String>pCollections());
-    PCollectionView<String> regexView = regex.apply(View.<String>asSingleton());
-    CountWords countWords = new CountWords(regexView);
-    PCollectionTuple luc = union.apply(countWords);
-    PCollection<Long> unique = luc.get(lowerCnts).apply(
-        ApproximateUnique.<KV<String, Long>>globally(16));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    Iterable<KV<String, Long>> actualLower = res.get(luc.get(lowerCnts));
-    Assert.assertEquals("are", actualLower.iterator().next().getKey());
-    Iterable<KV<String, Long>> actualUpper = res.get(luc.get(upperCnts));
-    Assert.assertEquals("Here", actualUpper.iterator().next().getKey());
-    Iterable<Long> actualUniqCount = res.get(unique);
-    Assert.assertEquals(9, (long) actualUniqCount.iterator().next());
-    int actualTotalWords = res.getAggregatorValue("totalWords", Integer.class);
-    Assert.assertEquals(18, actualTotalWords);
-    int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class);
-    Assert.assertEquals(6, actualMaxWordLength);
-    AggregatorValues<Integer> aggregatorValues = res.getAggregatorValues(countWords
-        .getTotalWordsAggregator());
-    Assert.assertEquals(18, Iterables.getOnlyElement(aggregatorValues.getValues()).intValue());
-
-    res.close();
-  }
-
-  /**
-   * A DoFn that tokenizes lines of text into individual words.
-   */
-  static class ExtractWordsFn extends DoFn<String, String> {
-
-    private final Aggregator<Integer, Integer> totalWords = createAggregator("totalWords",
-        new Sum.SumIntegerFn());
-    private final Aggregator<Integer, Integer> maxWordLength = createAggregator("maxWordLength",
-        new Max.MaxIntegerFn());
-    private final PCollectionView<String> regex;
-
-    ExtractWordsFn(PCollectionView<String> regex) {
-      this.regex = regex;
-    }
-
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] words = c.element().split(c.sideInput(regex));
-      for (String word : words) {
-        totalWords.addValue(1);
-        if (!word.isEmpty()) {
-          maxWordLength.addValue(word.length());
-          if (Character.isLowerCase(word.charAt(0))) {
-            c.output(word);
-          } else {
-            c.sideOutput(upper, word);
-          }
-        }
-      }
-    }
-  }
-
-  public static class CountWords extends PTransform<PCollection<String>, PCollectionTuple> {
-
-    private final PCollectionView<String> regex;
-    private final ExtractWordsFn extractWordsFn;
-
-    public CountWords(PCollectionView<String> regex) {
-      this.regex = regex;
-      this.extractWordsFn = new ExtractWordsFn(regex);
-    }
-
-    @Override
-    public PCollectionTuple apply(PCollection<String> lines) {
-      // Convert lines of text into individual words.
-      PCollectionTuple lowerUpper = lines
-          .apply(ParDo.of(extractWordsFn)
-              .withSideInputs(regex)
-              .withOutputTags(lower, TupleTagList.of(upper)));
-      lowerUpper.get(lower).setCoder(StringUtf8Coder.of());
-      lowerUpper.get(upper).setCoder(StringUtf8Coder.of());
-      PCollection<KV<String, Long>> lowerCounts = lowerUpper.get(lower).apply(Count
-          .<String>perElement());
-      PCollection<KV<String, Long>> upperCounts = lowerUpper.get(upper).apply(Count
-          .<String>perElement());
-      return PCollectionTuple
-          .of(lowerCnts, lowerCounts)
-          .and(upperCnts, upperCounts);
-    }
-
-    Aggregator<Integer, Integer> getTotalWordsAggregator() {
-      return extractWordsFn.totalWords;
-    }
-  }
-}


[08/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
new file mode 100644
index 0000000..836987f
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
@@ -0,0 +1,284 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaSparkContext;
+
+
+/**
+ * Evaluation context allows us to define how pipeline instructions.
+ */
+public class EvaluationContext implements EvaluationResult {
+  private final JavaSparkContext jsc;
+  private final Pipeline pipeline;
+  private final SparkRuntimeContext runtime;
+  private final Map<PValue, RDDHolder<?>> pcollections = new LinkedHashMap<>();
+  private final Set<RDDHolder<?>> leafRdds = new LinkedHashSet<>();
+  private final Set<PValue> multireads = new LinkedHashSet<>();
+  private final Map<PValue, Object> pobjects = new LinkedHashMap<>();
+  private final Map<PValue, Iterable<? extends WindowedValue<?>>> pview = new LinkedHashMap<>();
+  protected AppliedPTransform<?, ?, ?> currentTransform;
+
+  public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
+    this.jsc = jsc;
+    this.pipeline = pipeline;
+    this.runtime = new SparkRuntimeContext(jsc, pipeline);
+  }
+
+  /**
+   * Holds an RDD or values for deferred conversion to an RDD if needed. PCollections are
+   * sometimes created from a collection of objects (using RDD parallelize) and then
+   * only used to create View objects; in which case they do not need to be
+   * converted to bytes since they are not transferred across the network until they are
+   * broadcast.
+   */
+  private class RDDHolder<T> {
+
+    private Iterable<T> values;
+    private Coder<T> coder;
+    private JavaRDDLike<WindowedValue<T>, ?> rdd;
+
+    RDDHolder(Iterable<T> values, Coder<T> coder) {
+      this.values = values;
+      this.coder = coder;
+    }
+
+    RDDHolder(JavaRDDLike<WindowedValue<T>, ?> rdd) {
+      this.rdd = rdd;
+    }
+
+    JavaRDDLike<WindowedValue<T>, ?> getRDD() {
+      if (rdd == null) {
+        Iterable<WindowedValue<T>> windowedValues = Iterables.transform(values,
+            new Function<T, WindowedValue<T>>() {
+            @Override
+            public WindowedValue<T> apply(T t) {
+             // TODO: this is wrong if T is a TimestampedValue
+              return WindowedValue.valueInEmptyWindows(t);
+            }
+        });
+        WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder =
+            WindowedValue.getValueOnlyCoder(coder);
+        rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
+            .map(CoderHelpers.fromByteFunction(windowCoder));
+      }
+      return rdd;
+    }
+
+    Iterable<T> getValues(PCollection<T> pcollection) {
+      if (values == null) {
+        coder = pcollection.getCoder();
+        JavaRDDLike<byte[], ?> bytesRDD = rdd.map(WindowingHelpers.<T>unwindowFunction())
+            .map(CoderHelpers.toByteFunction(coder));
+        List<byte[]> clientBytes = bytesRDD.collect();
+        values = Iterables.transform(clientBytes, new Function<byte[], T>() {
+          @Override
+          public T apply(byte[] bytes) {
+            return CoderHelpers.fromByteArray(bytes, coder);
+          }
+        });
+      }
+      return values;
+    }
+
+    Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
+      return Iterables.transform(get(pcollection), new Function<T, WindowedValue<T>>() {
+        @Override
+        public WindowedValue<T> apply(T t) {
+          return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place?
+        }
+      });
+    }
+  }
+
+  protected JavaSparkContext getSparkContext() {
+    return jsc;
+  }
+
+  protected Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  protected SparkRuntimeContext getRuntimeContext() {
+    return runtime;
+  }
+
+  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
+    this.currentTransform = transform;
+  }
+
+  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
+    return currentTransform;
+  }
+
+  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
+    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
+        "can only be called with current transform");
+    @SuppressWarnings("unchecked")
+    I input = (I) currentTransform.getInput();
+    return input;
+  }
+
+  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
+    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
+        "can only be called with current transform");
+    @SuppressWarnings("unchecked")
+    O output = (O) currentTransform.getOutput();
+    return output;
+  }
+
+  protected  <T> void setOutputRDD(PTransform<?, ?> transform,
+      JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    setRDD((PValue) getOutput(transform), rdd);
+  }
+
+  protected  <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
+      Coder<T> coder) {
+    pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder));
+  }
+
+  void setPView(PValue view, Iterable<? extends WindowedValue<?>> value) {
+    pview.put(view, value);
+  }
+
+  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
+    PValue pvalue = (PValue) getOutput(transform);
+    return pcollections.containsKey(pvalue);
+  }
+
+  protected JavaRDDLike<?, ?> getRDD(PValue pvalue) {
+    RDDHolder<?> rddHolder = pcollections.get(pvalue);
+    JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
+    leafRdds.remove(rddHolder);
+    if (multireads.contains(pvalue)) {
+      // Ensure the RDD is marked as cached
+      rdd.rdd().cache();
+    } else {
+      multireads.add(pvalue);
+    }
+    return rdd;
+  }
+
+  protected <T> void setRDD(PValue pvalue, JavaRDDLike<WindowedValue<T>, ?> rdd) {
+    try {
+      rdd.rdd().setName(pvalue.getName());
+    } catch (IllegalStateException e) {
+      // name not set, ignore
+    }
+    RDDHolder<T> rddHolder = new RDDHolder<>(rdd);
+    pcollections.put(pvalue, rddHolder);
+    leafRdds.add(rddHolder);
+  }
+
+  JavaRDDLike<?, ?> getInputRDD(PTransform<? extends PInput, ?> transform) {
+    return getRDD((PValue) getInput(transform));
+  }
+
+
+  <T> Iterable<? extends WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
+    return pview.get(view);
+  }
+
+  /**
+   * Computes the outputs for all RDDs that are leaves in the DAG and do not have any
+   * actions (like saving to a file) registered on them (i.e. they are performed for side
+   * effects).
+   */
+  protected void computeOutputs() {
+    for (RDDHolder<?> rddHolder : leafRdds) {
+      JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
+      rdd.rdd().cache(); // cache so that any subsequent get() is cheap
+      rdd.count(); // force the RDD to be computed
+    }
+  }
+
+  @Override
+  public <T> T get(PValue value) {
+    if (pobjects.containsKey(value)) {
+      @SuppressWarnings("unchecked")
+      T result = (T) pobjects.get(value);
+      return result;
+    }
+    if (pcollections.containsKey(value)) {
+      JavaRDDLike<?, ?> rdd = pcollections.get(value).getRDD();
+      @SuppressWarnings("unchecked")
+      T res = (T) Iterables.getOnlyElement(rdd.collect());
+      pobjects.put(value, res);
+      return res;
+    }
+    throw new IllegalStateException("Cannot resolve un-known PObject: " + value);
+  }
+
+  @Override
+  public <T> T getAggregatorValue(String named, Class<T> resultType) {
+    return runtime.getAggregatorValue(named, resultType);
+  }
+
+  @Override
+  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
+      throws AggregatorRetrievalException {
+    return runtime.getAggregatorValues(aggregator);
+  }
+
+  @Override
+  public <T> Iterable<T> get(PCollection<T> pcollection) {
+    @SuppressWarnings("unchecked")
+    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
+    return rddHolder.getValues(pcollection);
+  }
+
+  <T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
+    @SuppressWarnings("unchecked")
+    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
+    return rddHolder.getWindowedValues(pcollection);
+  }
+
+  @Override
+  public void close() {
+    SparkContextFactory.stopSparkContext(jsc);
+  }
+
+  /** The runner is blocking. */
+  @Override
+  public State getState() {
+    return State.DONE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
new file mode 100644
index 0000000..4de97f6
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PValue;
+
+/**
+ * Interface for retrieving the result(s) of running a pipeline. Allows us to translate between
+ * {@code PObject<T>}s or {@code PCollection<T>}s and Ts or collections of Ts.
+ */
+public interface EvaluationResult extends PipelineResult {
+  /**
+   * Retrieves an iterable of results associated with the PCollection passed in.
+   *
+   * @param pcollection Collection we wish to translate.
+   * @param <T>         Type of elements contained in collection.
+   * @return Natively types result associated with collection.
+   */
+  <T> Iterable<T> get(PCollection<T> pcollection);
+
+  /**
+   * Retrieve an object of Type T associated with the PValue passed in.
+   *
+   * @param pval PValue to retrieve associated data for.
+   * @param <T>  Type of object to return.
+   * @return Native object.
+   */
+  <T> T get(PValue pval);
+
+  /**
+   * Retrieves the final value of the aggregator.
+   *
+   * @param aggName    name of aggregator.
+   * @param resultType Class of final result of aggregation.
+   * @param <T>        Type of final result of aggregation.
+   * @return Result of aggregation associated with specified name.
+   */
+  <T> T getAggregatorValue(String aggName, Class<T> resultType);
+
+  /**
+   * Releases any runtime resources, including distributed-execution contexts currently held by
+   * this EvaluationResult; once close() has been called,
+   * {@link EvaluationResult#get(PCollection)} might
+   * not work for subsequent calls.
+   */
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
new file mode 100644
index 0000000..968825b
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.joda.time.Instant;
+import scala.Tuple2;
+
+/**
+ * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enriching the
+ * underlying data with multiple TupleTags.
+ *
+ * @param <I> Input type for DoFunction.
+ * @param <O> Output type for DoFunction.
+ */
+class MultiDoFnFunction<I, O>
+    implements PairFlatMapFunction<Iterator<WindowedValue<I>>, TupleTag<?>, WindowedValue<?>> {
+  private final DoFn<I, O> mFunction;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final TupleTag<O> mMainOutputTag;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  MultiDoFnFunction(
+      DoFn<I, O> fn,
+      SparkRuntimeContext runtimeContext,
+      TupleTag<O> mainOutputTag,
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    this.mFunction = fn;
+    this.mRuntimeContext = runtimeContext;
+    this.mMainOutputTag = mainOutputTag;
+    this.mSideInputs = sideInputs;
+  }
+
+  @Override
+  public Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>>
+      call(Iterator<WindowedValue<I>> iter) throws Exception {
+    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
+    mFunction.startBundle(ctxt);
+    ctxt.setup();
+    return ctxt.getOutputIterable(iter, mFunction);
+  }
+
+  private class ProcCtxt extends SparkProcessContext<I, O, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
+        BroadcastHelper<?>> sideInputs) {
+      super(fn, runtimeContext, sideInputs);
+    }
+
+    @Override
+    public synchronized void output(O o) {
+      outputs.put(mMainOutputTag, windowedValue.withValue(o));
+    }
+
+    @Override
+    public synchronized void output(WindowedValue<O> o) {
+      outputs.put(mMainOutputTag, o);
+    }
+
+    @Override
+    public synchronized <T> void sideOutput(TupleTag<T> tag, T t) {
+      outputs.put(tag, windowedValue.withValue(t));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
+      outputs.put(tupleTag, WindowedValue.of(t, instant,
+          windowedValue.getWindows(), windowedValue.getPane()));
+    }
+
+    @Override
+    protected void clearOutput() {
+      outputs.clear();
+    }
+
+    @Override
+    protected Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> getOutputIterator() {
+      return Iterators.transform(outputs.entries().iterator(),
+          new Function<Map.Entry<TupleTag<?>, WindowedValue<?>>,
+              Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+        @Override
+        public Tuple2<TupleTag<?>, WindowedValue<?>> apply(Map.Entry<TupleTag<?>,
+            WindowedValue<?>> input) {
+          return new Tuple2<TupleTag<?>, WindowedValue<?>>(input.getKey(), input.getValue());
+        }
+      });
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
new file mode 100644
index 0000000..10b7369
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.serializer.KryoSerializer;
+
+final class SparkContextFactory {
+
+  /**
+   * If the property {@code dataflow.spark.test.reuseSparkContext} is set to
+   * {@code true} then the Spark context will be reused for dataflow pipelines.
+   * This property should only be enabled for tests.
+   */
+  static final String TEST_REUSE_SPARK_CONTEXT =
+      "dataflow.spark.test.reuseSparkContext";
+  private static JavaSparkContext sparkContext;
+  private static String sparkMaster;
+
+  private SparkContextFactory() {
+  }
+
+  static synchronized JavaSparkContext getSparkContext(String master, String appName) {
+    if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
+      if (sparkContext == null) {
+        sparkContext = createSparkContext(master, appName);
+        sparkMaster = master;
+      } else if (!master.equals(sparkMaster)) {
+        throw new IllegalArgumentException(String.format("Cannot reuse spark context " +
+                "with different spark master URL. Existing: %s, requested: %s.",
+            sparkMaster, master));
+      }
+      return sparkContext;
+    } else {
+      return createSparkContext(master, appName);
+    }
+  }
+
+  static synchronized void stopSparkContext(JavaSparkContext context) {
+    if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
+      context.stop();
+    }
+  }
+
+  private static JavaSparkContext createSparkContext(String master, String appName) {
+    SparkConf conf = new SparkConf();
+    conf.setMaster(master);
+    conf.setAppName(appName);
+    conf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
+    return new JavaSparkContext(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
new file mode 100644
index 0000000..913e5a1
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+
+/**
+ * Pipeline {@link SparkPipelineRunner.Evaluator} for Spark.
+ */
+public final class SparkPipelineEvaluator extends SparkPipelineRunner.Evaluator {
+
+  private final EvaluationContext ctxt;
+
+  public SparkPipelineEvaluator(EvaluationContext ctxt, SparkPipelineTranslator translator) {
+    super(translator);
+    this.ctxt = ctxt;
+  }
+
+  @Override
+  protected <PT extends PTransform<? super PInput, POutput>> void doVisitTransform(TransformTreeNode
+      node) {
+    @SuppressWarnings("unchecked")
+    PT transform = (PT) node.getTransform();
+    @SuppressWarnings("unchecked")
+    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
+    @SuppressWarnings("unchecked") TransformEvaluator<PT> evaluator =
+        (TransformEvaluator<PT>) translator.translate(transformClass);
+    LOG.info("Evaluating {}", transform);
+    AppliedPTransform<PInput, POutput, PT> appliedTransform =
+        AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform);
+    ctxt.setCurrentTransform(appliedTransform);
+    evaluator.evaluate(transform, ctxt);
+    ctxt.setCurrentTransform(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
new file mode 100644
index 0000000..1a5093b
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.StreamingOptions;
+
+public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions,
+                                              ApplicationNameOptions {
+  @Description("The url of the spark master to connect to, (e.g. spark://host:port, local[4]).")
+  @Default.String("local[1]")
+  String getSparkMaster();
+
+  void setSparkMaster(String master);
+
+  @Override
+  @Default.Boolean(false)
+  boolean isStreaming();
+
+  @Override
+  @Default.String("spark dataflow pipeline job")
+  String getAppName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
new file mode 100644
index 0000000..7b44ee4
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+
+public final class SparkPipelineOptionsFactory {
+  private SparkPipelineOptionsFactory() {
+  }
+
+  public static SparkPipelineOptions create() {
+    return PipelineOptionsFactory.as(SparkPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
new file mode 100644
index 0000000..9f7f8c1
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+
+public class SparkPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
+  @Override
+  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkPipelineOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
new file mode 100644
index 0000000..429750d
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -0,0 +1,252 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+
+import org.apache.beam.runners.spark.streaming.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.streaming.StreamingEvaluationContext;
+import org.apache.beam.runners.spark.streaming.StreamingTransformTranslator;
+import org.apache.beam.runners.spark.streaming.StreamingWindowPipelineDetector;
+
+import org.apache.spark.SparkException;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The SparkPipelineRunner translate operations defined on a pipeline to a representation
+ * executable by Spark, and then submitting the job to Spark to be executed. If we wanted to run
+ * a dataflow pipeline with the default options of a single threaded spark instance in local mode,
+ * we would do the following:
+ *
+ * {@code
+ * Pipeline p = [logic for pipeline creation]
+ * EvaluationResult result = SparkPipelineRunner.create().run(p);
+ * }
+ *
+ * To create a pipeline runner to run against a different spark cluster, with a custom master url
+ * we would do the following:
+ *
+ * {@code
+ * Pipeline p = [logic for pipeline creation]
+ * SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+ * options.setSparkMaster("spark://host:port");
+ * EvaluationResult result = SparkPipelineRunner.create(options).run(p);
+ * }
+ *
+ * To create a Spark streaming pipeline runner use {@link SparkStreamingPipelineOptions}
+ */
+public final class SparkPipelineRunner extends PipelineRunner<EvaluationResult> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPipelineRunner.class);
+  /**
+   * Options used in this pipeline runner.
+   */
+  private final SparkPipelineOptions mOptions;
+
+  /**
+   * Creates and returns a new SparkPipelineRunner with default options. In particular, against a
+   * spark instance running in local mode.
+   *
+   * @return A pipeline runner with default options.
+   */
+  public static SparkPipelineRunner create() {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    return new SparkPipelineRunner(options);
+  }
+
+  /**
+   * Creates and returns a new SparkPipelineRunner with specified options.
+   *
+   * @param options The SparkPipelineOptions to use when executing the job.
+   * @return A pipeline runner that will execute with specified options.
+   */
+  public static SparkPipelineRunner create(SparkPipelineOptions options) {
+    return new SparkPipelineRunner(options);
+  }
+
+  /**
+   * Creates and returns a new SparkPipelineRunner with specified options.
+   *
+   * @param options The PipelineOptions to use when executing the job.
+   * @return A pipeline runner that will execute with specified options.
+   */
+  public static SparkPipelineRunner fromOptions(PipelineOptions options) {
+    SparkPipelineOptions sparkOptions =
+        PipelineOptionsValidator.validate(SparkPipelineOptions.class, options);
+    return new SparkPipelineRunner(sparkOptions);
+  }
+
+  /**
+   * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single
+   * thread.
+   */
+  private SparkPipelineRunner(SparkPipelineOptions options) {
+    mOptions = options;
+  }
+
+
+  @Override
+  public EvaluationResult run(Pipeline pipeline) {
+    try {
+      // validate streaming configuration
+      if (mOptions.isStreaming() && !(mOptions instanceof SparkStreamingPipelineOptions)) {
+        throw new RuntimeException("A streaming job must be configured with " +
+            SparkStreamingPipelineOptions.class.getSimpleName() + ", found " +
+            mOptions.getClass().getSimpleName());
+      }
+      LOG.info("Executing pipeline using the SparkPipelineRunner.");
+      JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions
+              .getSparkMaster(), mOptions.getAppName());
+
+      if (mOptions.isStreaming()) {
+        SparkPipelineTranslator translator =
+                new StreamingTransformTranslator.Translator(new TransformTranslator.Translator());
+        // if streaming - fixed window should be defined on all UNBOUNDED inputs
+        StreamingWindowPipelineDetector streamingWindowPipelineDetector =
+            new StreamingWindowPipelineDetector(translator);
+        pipeline.traverseTopologically(streamingWindowPipelineDetector);
+        if (!streamingWindowPipelineDetector.isWindowing()) {
+          throw new IllegalStateException("Spark streaming pipeline must be windowed!");
+        }
+
+        Duration batchInterval = streamingWindowPipelineDetector.getBatchDuration();
+        LOG.info("Setting Spark streaming batchInterval to {} msec", batchInterval.milliseconds());
+        EvaluationContext ctxt = createStreamingEvaluationContext(jsc, pipeline, batchInterval);
+
+        pipeline.traverseTopologically(new SparkPipelineEvaluator(ctxt, translator));
+        ctxt.computeOutputs();
+
+        LOG.info("Streaming pipeline construction complete. Starting execution..");
+        ((StreamingEvaluationContext) ctxt).getStreamingContext().start();
+
+        return ctxt;
+      } else {
+        EvaluationContext ctxt = new EvaluationContext(jsc, pipeline);
+        SparkPipelineTranslator translator = new TransformTranslator.Translator();
+        pipeline.traverseTopologically(new SparkPipelineEvaluator(ctxt, translator));
+        ctxt.computeOutputs();
+
+        LOG.info("Pipeline execution complete.");
+
+        return ctxt;
+      }
+    } catch (Exception e) {
+      // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler
+      // won't let you catch something that is not declared, so we can't catch
+      // SparkException here. Instead we do an instanceof check.
+      // Then we find the cause by seeing if it's a user exception (wrapped by our
+      // SparkProcessException), or just use the SparkException cause.
+      if (e instanceof SparkException && e.getCause() != null) {
+        if (e.getCause() instanceof SparkProcessContext.SparkProcessException &&
+                e.getCause().getCause() != null) {
+          throw new RuntimeException(e.getCause().getCause());
+        } else {
+          throw new RuntimeException(e.getCause());
+        }
+      }
+      // otherwise just wrap in a RuntimeException
+      throw new RuntimeException(e);
+    }
+  }
+
+  private EvaluationContext
+      createStreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
+      Duration batchDuration) {
+    SparkStreamingPipelineOptions streamingOptions = (SparkStreamingPipelineOptions) mOptions;
+    JavaStreamingContext jssc = new JavaStreamingContext(jsc, batchDuration);
+    return new StreamingEvaluationContext(jsc, pipeline, jssc, streamingOptions.getTimeout());
+  }
+
+  public abstract static class Evaluator implements Pipeline.PipelineVisitor {
+    protected static final Logger LOG = LoggerFactory.getLogger(Evaluator.class);
+
+    protected final SparkPipelineTranslator translator;
+
+    protected Evaluator(SparkPipelineTranslator translator) {
+      this.translator = translator;
+    }
+
+    // Set upon entering a composite node which can be directly mapped to a single
+    // TransformEvaluator.
+    private TransformTreeNode currentTranslatedCompositeNode;
+
+    /**
+     * If true, we're currently inside a subtree of a composite node which directly maps to a
+     * single
+     * TransformEvaluator; children nodes are ignored, and upon post-visiting the translated
+     * composite node, the associated TransformEvaluator will be visited.
+     */
+    private boolean inTranslatedCompositeNode() {
+      return currentTranslatedCompositeNode != null;
+    }
+
+    @Override
+    public void enterCompositeTransform(TransformTreeNode node) {
+      if (!inTranslatedCompositeNode() && node.getTransform() != null) {
+        @SuppressWarnings("unchecked")
+        Class<PTransform<?, ?>> transformClass =
+            (Class<PTransform<?, ?>>) node.getTransform().getClass();
+        if (translator.hasTranslation(transformClass)) {
+          LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName());
+          LOG.debug("Composite transform class: '{}'", transformClass);
+          currentTranslatedCompositeNode = node;
+        }
+      }
+    }
+
+    @Override
+    public void leaveCompositeTransform(TransformTreeNode node) {
+      // NB: We depend on enterCompositeTransform and leaveCompositeTransform providing 'node'
+      // objects for which Object.equals() returns true iff they are the same logical node
+      // within the tree.
+      if (inTranslatedCompositeNode() && node.equals(currentTranslatedCompositeNode)) {
+        LOG.info("Post-visiting directly-translatable composite transform: '{}'",
+                node.getFullName());
+        doVisitTransform(node);
+        currentTranslatedCompositeNode = null;
+      }
+    }
+
+    @Override
+    public void visitTransform(TransformTreeNode node) {
+      if (inTranslatedCompositeNode()) {
+        LOG.info("Skipping '{}'; already in composite transform.", node.getFullName());
+        return;
+      }
+      doVisitTransform(node);
+    }
+
+    protected abstract <PT extends PTransform<? super PInput, POutput>> void
+        doVisitTransform(TransformTreeNode node);
+
+    @Override
+    public void visitValue(PValue value, TransformTreeNode producer) {
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
new file mode 100644
index 0000000..9a84370
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
+import com.google.common.collect.ImmutableList;
+
+public class SparkPipelineRunnerRegistrar implements PipelineRunnerRegistrar {
+  @Override
+  public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+    return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
new file mode 100644
index 0000000..e45491a
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+/**
+ * Translator to support translation between Dataflow transformations and Spark transformations.
+ */
+public interface SparkPipelineTranslator {
+
+  boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz);
+
+  <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
new file mode 100644
index 0000000..c634152
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
@@ -0,0 +1,257 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkProcessContext<I, O, V> extends DoFn<I, O>.ProcessContext {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class);
+
+  private final DoFn<I, O> fn;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  protected WindowedValue<I> windowedValue;
+
+  SparkProcessContext(DoFn<I, O> fn,
+      SparkRuntimeContext runtime,
+      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    fn.super();
+    this.fn = fn;
+    this.mRuntimeContext = runtime;
+    this.mSideInputs = sideInputs;
+  }
+
+  void setup() {
+    setupDelegateAggregators();
+  }
+
+  @Override
+  public PipelineOptions getPipelineOptions() {
+    return mRuntimeContext.getPipelineOptions();
+  }
+
+  @Override
+  public <T> T sideInput(PCollectionView<T> view) {
+    @SuppressWarnings("unchecked")
+    BroadcastHelper<Iterable<WindowedValue<?>>> broadcastHelper =
+        (BroadcastHelper<Iterable<WindowedValue<?>>>) mSideInputs.get(view.getTagInternal());
+    Iterable<WindowedValue<?>> contents = broadcastHelper.getValue();
+    return view.fromIterableInternal(contents);
+  }
+
+  @Override
+  public abstract void output(O output);
+
+  public abstract void output(WindowedValue<O> output);
+
+  @Override
+  public <T> void sideOutput(TupleTag<T> tupleTag, T t) {
+    String message = "sideOutput is an unsupported operation for doFunctions, use a " +
+        "MultiDoFunction instead.";
+    LOG.warn(message);
+    throw new UnsupportedOperationException(message);
+  }
+
+  @Override
+  public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
+    String message =
+        "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " +
+            "MultiDoFunction instead.";
+    LOG.warn(message);
+    throw new UnsupportedOperationException(message);
+  }
+
+  @Override
+  public <AI, AO> Aggregator<AI, AO> createAggregatorInternal(
+      String named,
+      Combine.CombineFn<AI, ?, AO> combineFn) {
+    return mRuntimeContext.createAggregator(named, combineFn);
+  }
+
+  @Override
+  public I element() {
+    return windowedValue.getValue();
+  }
+
+  @Override
+  public void outputWithTimestamp(O output, Instant timestamp) {
+    output(WindowedValue.of(output, timestamp,
+        windowedValue.getWindows(), windowedValue.getPane()));
+  }
+
+  @Override
+  public Instant timestamp() {
+    return windowedValue.getTimestamp();
+  }
+
+  @Override
+  public BoundedWindow window() {
+    if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+      throw new UnsupportedOperationException(
+          "window() is only available in the context of a DoFn marked as RequiresWindow.");
+    }
+    return Iterables.getOnlyElement(windowedValue.getWindows());
+  }
+
+  @Override
+  public PaneInfo pane() {
+    return windowedValue.getPane();
+  }
+
+  @Override
+  public WindowingInternals<I, O> windowingInternals() {
+    return new WindowingInternals<I, O>() {
+
+      @Override
+      public Collection<? extends BoundedWindow> windows() {
+        return windowedValue.getWindows();
+      }
+
+      @Override
+      public void outputWindowedValue(O output, Instant timestamp, Collection<?
+          extends BoundedWindow> windows, PaneInfo paneInfo) {
+        output(WindowedValue.of(output, timestamp, windows, paneInfo));
+      }
+
+      @Override
+      public StateInternals stateInternals() {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#stateInternals() is not yet supported.");
+      }
+
+      @Override
+      public TimerInternals timerInternals() {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#timerInternals() is not yet supported.");
+      }
+
+      @Override
+      public PaneInfo pane() {
+        return windowedValue.getPane();
+      }
+
+      @Override
+      public <T> void writePCollectionViewData(TupleTag<?> tag,
+          Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#writePCollectionViewData() is not yet supported.");
+      }
+
+      @Override
+      public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+        throw new UnsupportedOperationException(
+            "WindowingInternals#sideInput() is not yet supported.");
+      }
+    };
+  }
+
+  protected abstract void clearOutput();
+  protected abstract Iterator<V> getOutputIterator();
+
+  protected Iterable<V> getOutputIterable(final Iterator<WindowedValue<I>> iter,
+      final DoFn<I, O> doFn) {
+    return new Iterable<V>() {
+      @Override
+      public Iterator<V> iterator() {
+        return new ProcCtxtIterator(iter, doFn);
+      }
+    };
+  }
+
+  private class ProcCtxtIterator extends AbstractIterator<V> {
+
+    private final Iterator<WindowedValue<I>> inputIterator;
+    private final DoFn<I, O> doFn;
+    private Iterator<V> outputIterator;
+    private boolean calledFinish;
+
+    ProcCtxtIterator(Iterator<WindowedValue<I>> iterator, DoFn<I, O> doFn) {
+      this.inputIterator = iterator;
+      this.doFn = doFn;
+      this.outputIterator = getOutputIterator();
+    }
+
+    @Override
+    protected V computeNext() {
+      // Process each element from the (input) iterator, which produces, zero, one or more
+      // output elements (of type V) in the output iterator. Note that the output
+      // collection (and iterator) is reset between each call to processElement, so the
+      // collection only holds the output values for each call to processElement, rather
+      // than for the whole partition (which would use too much memory).
+      while (true) {
+        if (outputIterator.hasNext()) {
+          return outputIterator.next();
+        } else if (inputIterator.hasNext()) {
+          clearOutput();
+          windowedValue = inputIterator.next();
+          try {
+            doFn.processElement(SparkProcessContext.this);
+          } catch (Exception e) {
+            throw new SparkProcessException(e);
+          }
+          outputIterator = getOutputIterator();
+        } else {
+          // no more input to consume, but finishBundle can produce more output
+          if (!calledFinish) {
+            clearOutput();
+            try {
+              calledFinish = true;
+              doFn.finishBundle(SparkProcessContext.this);
+            } catch (Exception e) {
+              throw new SparkProcessException(e);
+            }
+            outputIterator = getOutputIterator();
+            continue; // try to consume outputIterator from start of loop
+          }
+          return endOfData();
+        }
+      }
+    }
+  }
+
+  static class SparkProcessException extends RuntimeException {
+    SparkProcessException(Throwable t) {
+      super(t);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
new file mode 100644
index 0000000..da48ad7
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
@@ -0,0 +1,214 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Max;
+import com.google.cloud.dataflow.sdk.transforms.Min;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.beam.runners.spark.aggregators.AggAccumParam;
+import org.apache.beam.runners.spark.aggregators.NamedAggregators;
+
+import org.apache.spark.Accumulator;
+import org.apache.spark.api.java.JavaSparkContext;
+
+
+/**
+ * The SparkRuntimeContext allows us to define useful features on the client side before our
+ * data flow program is launched.
+ */
+public class SparkRuntimeContext implements Serializable {
+  /**
+   * An accumulator that is a map from names to aggregators.
+   */
+  private final Accumulator<NamedAggregators> accum;
+
+  private final String serializedPipelineOptions;
+
+  /**
+   * Map fo names to dataflow aggregators.
+   */
+  private final Map<String, Aggregator<?, ?>> aggregators = new HashMap<>();
+  private transient CoderRegistry coderRegistry;
+
+  SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) {
+    this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam());
+    this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions());
+  }
+
+  private static String serializePipelineOptions(PipelineOptions pipelineOptions) {
+    try {
+      return new ObjectMapper().writeValueAsString(pipelineOptions);
+    } catch (JsonProcessingException e) {
+      throw new IllegalStateException("Failed to serialize the pipeline options.", e);
+    }
+  }
+
+  private static PipelineOptions deserializePipelineOptions(String serializedPipelineOptions) {
+    try {
+      return new ObjectMapper().readValue(serializedPipelineOptions, PipelineOptions.class);
+    } catch (IOException e) {
+      throw new IllegalStateException("Failed to deserialize the pipeline options.", e);
+    }
+  }
+
+  /**
+   * Retrieves corresponding value of an aggregator.
+   *
+   * @param aggregatorName Name of the aggregator to retrieve the value of.
+   * @param typeClass      Type class of value to be retrieved.
+   * @param <T>            Type of object to be returned.
+   * @return The value of the aggregator.
+   */
+  public <T> T getAggregatorValue(String aggregatorName, Class<T> typeClass) {
+    return accum.value().getValue(aggregatorName, typeClass);
+  }
+
+  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
+    @SuppressWarnings("unchecked")
+    Class<T> aggValueClass = (Class<T>) aggregator.getCombineFn().getOutputType().getRawType();
+    final T aggregatorValue = getAggregatorValue(aggregator.getName(), aggValueClass);
+    return new AggregatorValues<T>() {
+      @Override
+      public Collection<T> getValues() {
+        return ImmutableList.of(aggregatorValue);
+      }
+
+      @Override
+      public Map<String, T> getValuesAtSteps() {
+        throw new UnsupportedOperationException("getValuesAtSteps is not supported.");
+      }
+    };
+  }
+
+  public synchronized PipelineOptions getPipelineOptions() {
+    return deserializePipelineOptions(serializedPipelineOptions);
+  }
+
+  /**
+   * Creates and aggregator and associates it with the specified name.
+   *
+   * @param named     Name of aggregator.
+   * @param combineFn Combine function used in aggregation.
+   * @param <IN>      Type of inputs to aggregator.
+   * @param <INTER>   Intermediate data type
+   * @param <OUT>     Type of aggregator outputs.
+   * @return Specified aggregator
+   */
+  public synchronized <IN, INTER, OUT> Aggregator<IN, OUT> createAggregator(
+      String named,
+      Combine.CombineFn<? super IN, INTER, OUT> combineFn) {
+    @SuppressWarnings("unchecked")
+    Aggregator<IN, OUT> aggregator = (Aggregator<IN, OUT>) aggregators.get(named);
+    if (aggregator == null) {
+      @SuppressWarnings("unchecked")
+      NamedAggregators.CombineFunctionState<IN, INTER, OUT> state =
+          new NamedAggregators.CombineFunctionState<>(
+              (Combine.CombineFn<IN, INTER, OUT>) combineFn,
+              (Coder<IN>) getCoder(combineFn),
+              this);
+      accum.add(new NamedAggregators(named, state));
+      aggregator = new SparkAggregator<>(named, state);
+      aggregators.put(named, aggregator);
+    }
+    return aggregator;
+  }
+
+  public CoderRegistry getCoderRegistry() {
+    if (coderRegistry == null) {
+      coderRegistry = new CoderRegistry();
+      coderRegistry.registerStandardCoders();
+    }
+    return coderRegistry;
+  }
+
+  private Coder<?> getCoder(Combine.CombineFn<?, ?, ?> combiner) {
+    try {
+      if (combiner.getClass() == Sum.SumIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Sum.SumLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Sum.SumDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else if (combiner.getClass() == Min.MinIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Min.MinLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Min.MinDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else if (combiner.getClass() == Max.MaxIntegerFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
+      } else if (combiner.getClass() == Max.MaxLongFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
+      } else if (combiner.getClass() == Max.MaxDoubleFn.class) {
+        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
+      } else {
+        throw new IllegalArgumentException("unsupported combiner in Aggregator: "
+            + combiner.getClass().getName());
+      }
+    } catch (CannotProvideCoderException e) {
+      throw new IllegalStateException("Could not determine default coder for combiner", e);
+    }
+  }
+
+  /**
+   * Initialize spark aggregators exactly once.
+   *
+   * @param <IN> Type of element fed in to aggregator.
+   */
+  private static class SparkAggregator<IN, OUT> implements Aggregator<IN, OUT>, Serializable {
+    private final String name;
+    private final NamedAggregators.State<IN, ?, OUT> state;
+
+    SparkAggregator(String name, NamedAggregators.State<IN, ?, OUT> state) {
+      this.name = name;
+      this.state = state;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public void addValue(IN elem) {
+      state.update(elem);
+    }
+
+    @Override
+    public Combine.CombineFn<IN, ?, OUT> getCombineFn() {
+      return state.getCombineFn();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
new file mode 100644
index 0000000..8aaceeb
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import java.io.Serializable;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+public interface TransformEvaluator<PT extends PTransform<?, ?>> extends Serializable {
+  void evaluate(PT transform, EvaluationContext context);
+}


[02/23] incubator-beam git commit: [BEAM-11] extractOutput() should not return null

Posted by am...@apache.org.
[BEAM-11] extractOutput() should not return null


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

Branch: refs/heads/master
Commit: 7e9aab0651a56772ca585cb34ebfed76497c0e79
Parents: c5c7df6
Author: Sela <an...@paypal.com>
Authored: Sat Mar 12 17:27:38 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:38:26 2016 +0200

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/spark/CombineGloballyTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7e9aab06/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
index e36babe..49e68f5 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
@@ -73,7 +73,7 @@ public class CombineGloballyTest {
 
     @Override
     public String extractOutput(StringBuilder accumulator) {
-      return accumulator.toString();
+      return accumulator != null ? accumulator.toString(): "";
     }
 
     private static StringBuilder combine(StringBuilder accum, String datum) {


[19/23] incubator-beam git commit: [BEAM-11] second iteration of package reorganisation

Posted by am...@apache.org.
[BEAM-11] second iteration of package reorganisation


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

Branch: refs/heads/master
Commit: eb0341d4df6a14d8d815ebfc51ffc0254923a8fa
Parents: 3980f44
Author: Sela <an...@paypal.com>
Authored: Mon Mar 14 23:48:05 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:40:18 2016 +0200

----------------------------------------------------------------------
 .../apache/beam/runners/spark/DoFnFunction.java |  97 ---
 .../beam/runners/spark/EvaluationContext.java   | 287 -------
 .../beam/runners/spark/MultiDoFnFunction.java   | 119 ---
 .../beam/runners/spark/SparkContextFactory.java |  69 --
 .../runners/spark/SparkPipelineEvaluator.java   |  55 --
 .../spark/SparkPipelineOptionsFactory.java      |  30 -
 .../spark/SparkPipelineOptionsRegistrar.java    |  30 -
 .../beam/runners/spark/SparkPipelineRunner.java |   8 +-
 .../spark/SparkPipelineRunnerRegistrar.java     |  30 -
 .../runners/spark/SparkPipelineTranslator.java  |  30 -
 .../beam/runners/spark/SparkProcessContext.java | 262 ------
 .../beam/runners/spark/SparkRuntimeContext.java | 217 -----
 .../spark/SparkStreamingPipelineOptions.java    |  41 +
 .../beam/runners/spark/TransformEvaluator.java  |  27 -
 .../beam/runners/spark/TransformTranslator.java | 808 -------------------
 .../beam/runners/spark/WindowingHelpers.java    |  62 --
 .../spark/aggregators/NamedAggregators.java     |   2 +-
 .../SparkStreamingPipelineOptions.java          |  43 -
 .../SparkStreamingPipelineOptionsFactory.java   |  30 -
 .../SparkStreamingPipelineOptionsRegistrar.java |  31 -
 .../streaming/StreamingEvaluationContext.java   | 229 ------
 .../streaming/StreamingTransformTranslator.java | 418 ----------
 .../StreamingWindowPipelineDetector.java        | 104 ---
 .../runners/spark/translation/DoFnFunction.java |  97 +++
 .../spark/translation/EvaluationContext.java    | 288 +++++++
 .../spark/translation/MultiDoFnFunction.java    | 119 +++
 .../spark/translation/SparkContextFactory.java  |  69 ++
 .../translation/SparkPipelineEvaluator.java     |  56 ++
 .../SparkPipelineOptionsFactory.java            |  31 +
 .../SparkPipelineOptionsRegistrar.java          |  31 +
 .../SparkPipelineRunnerRegistrar.java           |  31 +
 .../translation/SparkPipelineTranslator.java    |  30 +
 .../spark/translation/SparkProcessContext.java  | 262 ++++++
 .../spark/translation/SparkRuntimeContext.java  | 217 +++++
 .../spark/translation/TransformEvaluator.java   |  27 +
 .../spark/translation/TransformTranslator.java  | 808 +++++++++++++++++++
 .../spark/translation/WindowingHelpers.java     |  62 ++
 .../SparkStreamingPipelineOptionsFactory.java   |  31 +
 .../SparkStreamingPipelineOptionsRegistrar.java |  32 +
 .../streaming/StreamingEvaluationContext.java   | 229 ++++++
 .../streaming/StreamingTransformTranslator.java | 418 ++++++++++
 .../StreamingWindowPipelineDetector.java        | 104 +++
 ...ataflow.sdk.options.PipelineOptionsRegistrar |   4 +-
 ...dataflow.sdk.runners.PipelineRunnerRegistrar |   2 +-
 .../beam/runners/spark/CombineGloballyTest.java |  91 ---
 .../beam/runners/spark/CombinePerKeyTest.java   |  68 --
 .../apache/beam/runners/spark/DeDupTest.java    |   1 +
 .../beam/runners/spark/DoFnOutputTest.java      |  61 --
 .../beam/runners/spark/EmptyInputTest.java      |   1 +
 .../runners/spark/MultiOutputWordCountTest.java | 135 ----
 .../beam/runners/spark/SerializationTest.java   | 180 -----
 .../beam/runners/spark/SideEffectsTest.java     |  79 --
 .../beam/runners/spark/SimpleWordCountTest.java |   1 +
 .../spark/TestSparkPipelineOptionsFactory.java  |  37 -
 .../runners/spark/TransformTranslatorTest.java  |  98 ---
 .../runners/spark/WindowedWordCountTest.java    |  67 --
 .../beam/runners/spark/io/NumShardsTest.java    |   2 +-
 .../spark/streaming/FlattenStreamingTest.java   |  87 --
 .../spark/streaming/KafkaStreamingTest.java     | 139 ----
 .../streaming/SimpleStreamingWordCountTest.java |  76 --
 .../utils/DataflowAssertStreaming.java          |  42 -
 .../streaming/utils/EmbeddedKafkaCluster.java   | 317 --------
 .../spark/translation/CombineGloballyTest.java  |  94 +++
 .../spark/translation/CombinePerKeyTest.java    |  70 ++
 .../spark/translation/DoFnOutputTest.java       |  64 ++
 .../translation/MultiOutputWordCountTest.java   | 137 ++++
 .../spark/translation/SerializationTest.java    | 183 +++++
 .../spark/translation/SideEffectsTest.java      |  81 ++
 .../TestSparkPipelineOptionsFactory.java        |  38 +
 .../translation/TransformTranslatorTest.java    |  99 +++
 .../translation/WindowedWordCountTest.java      |  71 ++
 .../streaming/FlattenStreamingTest.java         |  88 ++
 .../streaming/KafkaStreamingTest.java           | 140 ++++
 .../streaming/SimpleStreamingWordCountTest.java |  77 ++
 .../utils/DataflowAssertStreaming.java          |  42 +
 .../streaming/utils/EmbeddedKafkaCluster.java   | 317 ++++++++
 76 files changed, 4496 insertions(+), 4464 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
deleted file mode 100644
index e5d4542..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.apache.beam.runners.spark.util.BroadcastHelper;
-import org.apache.spark.api.java.function.FlatMapFunction;
-
-/**
- * Dataflow's Do functions correspond to Spark's FlatMap functions.
- *
- * @param <I> Input element type.
- * @param <O> Output element type.
- */
-public class DoFnFunction<I, O> implements FlatMapFunction<Iterator<WindowedValue<I>>,
-    WindowedValue<O>> {
-  private final DoFn<I, O> mFunction;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  /**
-   * @param fn         DoFunction to be wrapped.
-   * @param runtime    Runtime to apply function in.
-   * @param sideInputs Side inputs used in DoFunction.
-   */
-  public DoFnFunction(DoFn<I, O> fn,
-               SparkRuntimeContext runtime,
-               Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    this.mFunction = fn;
-    this.mRuntimeContext = runtime;
-    this.mSideInputs = sideInputs;
-  }
-
-  @Override
-  public Iterable<WindowedValue<O>> call(Iterator<WindowedValue<I>> iter) throws
-      Exception {
-    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
-    ctxt.setup();
-    mFunction.startBundle(ctxt);
-    return ctxt.getOutputIterable(iter, mFunction);
-  }
-
-  private class ProcCtxt extends SparkProcessContext<I, O, WindowedValue<O>> {
-
-    private final List<WindowedValue<O>> outputs = new LinkedList<>();
-
-    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
-        BroadcastHelper<?>> sideInputs) {
-      super(fn, runtimeContext, sideInputs);
-    }
-
-    @Override
-    public synchronized void output(O o) {
-      outputs.add(windowedValue != null ? windowedValue.withValue(o) :
-          WindowedValue.valueInEmptyWindows(o));
-    }
-
-    @Override
-    public synchronized void output(WindowedValue<O> o) {
-      outputs.add(o);
-    }
-
-    @Override
-    protected void clearOutput() {
-      outputs.clear();
-    }
-
-    @Override
-    protected Iterator<WindowedValue<O>> getOutputIterator() {
-      return outputs.iterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
deleted file mode 100644
index ad49528..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationContext.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import org.apache.beam.runners.spark.coders.CoderHelpers;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-
-
-/**
- * Evaluation context allows us to define how pipeline instructions.
- */
-public class EvaluationContext implements EvaluationResult {
-  private final JavaSparkContext jsc;
-  private final Pipeline pipeline;
-  private final SparkRuntimeContext runtime;
-  private final Map<PValue, RDDHolder<?>> pcollections = new LinkedHashMap<>();
-  private final Set<RDDHolder<?>> leafRdds = new LinkedHashSet<>();
-  private final Set<PValue> multireads = new LinkedHashSet<>();
-  private final Map<PValue, Object> pobjects = new LinkedHashMap<>();
-  private final Map<PValue, Iterable<? extends WindowedValue<?>>> pview = new LinkedHashMap<>();
-  protected AppliedPTransform<?, ?, ?> currentTransform;
-
-  public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
-    this.jsc = jsc;
-    this.pipeline = pipeline;
-    this.runtime = new SparkRuntimeContext(jsc, pipeline);
-  }
-
-  /**
-   * Holds an RDD or values for deferred conversion to an RDD if needed. PCollections are
-   * sometimes created from a collection of objects (using RDD parallelize) and then
-   * only used to create View objects; in which case they do not need to be
-   * converted to bytes since they are not transferred across the network until they are
-   * broadcast.
-   */
-  private class RDDHolder<T> {
-
-    private Iterable<T> values;
-    private Coder<T> coder;
-    private JavaRDDLike<WindowedValue<T>, ?> rdd;
-
-    RDDHolder(Iterable<T> values, Coder<T> coder) {
-      this.values = values;
-      this.coder = coder;
-    }
-
-    RDDHolder(JavaRDDLike<WindowedValue<T>, ?> rdd) {
-      this.rdd = rdd;
-    }
-
-    JavaRDDLike<WindowedValue<T>, ?> getRDD() {
-      if (rdd == null) {
-        Iterable<WindowedValue<T>> windowedValues = Iterables.transform(values,
-            new Function<T, WindowedValue<T>>() {
-            @Override
-            public WindowedValue<T> apply(T t) {
-             // TODO: this is wrong if T is a TimestampedValue
-              return WindowedValue.valueInEmptyWindows(t);
-            }
-        });
-        WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder =
-            WindowedValue.getValueOnlyCoder(coder);
-        rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
-            .map(CoderHelpers.fromByteFunction(windowCoder));
-      }
-      return rdd;
-    }
-
-    Iterable<T> getValues(PCollection<T> pcollection) {
-      if (values == null) {
-        coder = pcollection.getCoder();
-        JavaRDDLike<byte[], ?> bytesRDD = rdd.map(WindowingHelpers.<T>unwindowFunction())
-            .map(CoderHelpers.toByteFunction(coder));
-        List<byte[]> clientBytes = bytesRDD.collect();
-        values = Iterables.transform(clientBytes, new Function<byte[], T>() {
-          @Override
-          public T apply(byte[] bytes) {
-            return CoderHelpers.fromByteArray(bytes, coder);
-          }
-        });
-      }
-      return values;
-    }
-
-    Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
-      return Iterables.transform(get(pcollection), new Function<T, WindowedValue<T>>() {
-        @Override
-        public WindowedValue<T> apply(T t) {
-          return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place?
-        }
-      });
-    }
-  }
-
-  protected JavaSparkContext getSparkContext() {
-    return jsc;
-  }
-
-  protected Pipeline getPipeline() {
-    return pipeline;
-  }
-
-  protected SparkRuntimeContext getRuntimeContext() {
-    return runtime;
-  }
-
-  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
-    this.currentTransform = transform;
-  }
-
-  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
-    return currentTransform;
-  }
-
-  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
-    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-        "can only be called with current transform");
-    @SuppressWarnings("unchecked")
-    I input = (I) currentTransform.getInput();
-    return input;
-  }
-
-  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
-    checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-        "can only be called with current transform");
-    @SuppressWarnings("unchecked")
-    O output = (O) currentTransform.getOutput();
-    return output;
-  }
-
-  protected  <T> void setOutputRDD(PTransform<?, ?> transform,
-      JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    setRDD((PValue) getOutput(transform), rdd);
-  }
-
-  protected  <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
-      Coder<T> coder) {
-    pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder));
-  }
-
-  void setPView(PValue view, Iterable<? extends WindowedValue<?>> value) {
-    pview.put(view, value);
-  }
-
-  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
-    PValue pvalue = (PValue) getOutput(transform);
-    return pcollections.containsKey(pvalue);
-  }
-
-  protected JavaRDDLike<?, ?> getRDD(PValue pvalue) {
-    RDDHolder<?> rddHolder = pcollections.get(pvalue);
-    JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
-    leafRdds.remove(rddHolder);
-    if (multireads.contains(pvalue)) {
-      // Ensure the RDD is marked as cached
-      rdd.rdd().cache();
-    } else {
-      multireads.add(pvalue);
-    }
-    return rdd;
-  }
-
-  protected <T> void setRDD(PValue pvalue, JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    try {
-      rdd.rdd().setName(pvalue.getName());
-    } catch (IllegalStateException e) {
-      // name not set, ignore
-    }
-    RDDHolder<T> rddHolder = new RDDHolder<>(rdd);
-    pcollections.put(pvalue, rddHolder);
-    leafRdds.add(rddHolder);
-  }
-
-  JavaRDDLike<?, ?> getInputRDD(PTransform<? extends PInput, ?> transform) {
-    return getRDD((PValue) getInput(transform));
-  }
-
-
-  <T> Iterable<? extends WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
-    return pview.get(view);
-  }
-
-  /**
-   * Computes the outputs for all RDDs that are leaves in the DAG and do not have any
-   * actions (like saving to a file) registered on them (i.e. they are performed for side
-   * effects).
-   */
-  protected void computeOutputs() {
-    for (RDDHolder<?> rddHolder : leafRdds) {
-      JavaRDDLike<?, ?> rdd = rddHolder.getRDD();
-      rdd.rdd().cache(); // cache so that any subsequent get() is cheap
-      rdd.count(); // force the RDD to be computed
-    }
-  }
-
-  @Override
-  public <T> T get(PValue value) {
-    if (pobjects.containsKey(value)) {
-      @SuppressWarnings("unchecked")
-      T result = (T) pobjects.get(value);
-      return result;
-    }
-    if (pcollections.containsKey(value)) {
-      JavaRDDLike<?, ?> rdd = pcollections.get(value).getRDD();
-      @SuppressWarnings("unchecked")
-      T res = (T) Iterables.getOnlyElement(rdd.collect());
-      pobjects.put(value, res);
-      return res;
-    }
-    throw new IllegalStateException("Cannot resolve un-known PObject: " + value);
-  }
-
-  @Override
-  public <T> T getAggregatorValue(String named, Class<T> resultType) {
-    return runtime.getAggregatorValue(named, resultType);
-  }
-
-  @Override
-  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-      throws AggregatorRetrievalException {
-    return runtime.getAggregatorValues(aggregator);
-  }
-
-  @Override
-  public <T> Iterable<T> get(PCollection<T> pcollection) {
-    @SuppressWarnings("unchecked")
-    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
-    return rddHolder.getValues(pcollection);
-  }
-
-  <T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
-    @SuppressWarnings("unchecked")
-    RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
-    return rddHolder.getWindowedValues(pcollection);
-  }
-
-  @Override
-  public void close() {
-    SparkContextFactory.stopSparkContext(jsc);
-  }
-
-  /** The runner is blocking. */
-  @Override
-  public State getState() {
-    return State.DONE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
deleted file mode 100644
index 47433a6..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/MultiDoFnFunction.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Function;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.LinkedListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.beam.runners.spark.util.BroadcastHelper;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-import org.joda.time.Instant;
-import scala.Tuple2;
-
-/**
- * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enriching the
- * underlying data with multiple TupleTags.
- *
- * @param <I> Input type for DoFunction.
- * @param <O> Output type for DoFunction.
- */
-class MultiDoFnFunction<I, O>
-    implements PairFlatMapFunction<Iterator<WindowedValue<I>>, TupleTag<?>, WindowedValue<?>> {
-  private final DoFn<I, O> mFunction;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final TupleTag<O> mMainOutputTag;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  MultiDoFnFunction(
-      DoFn<I, O> fn,
-      SparkRuntimeContext runtimeContext,
-      TupleTag<O> mainOutputTag,
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    this.mFunction = fn;
-    this.mRuntimeContext = runtimeContext;
-    this.mMainOutputTag = mainOutputTag;
-    this.mSideInputs = sideInputs;
-  }
-
-  @Override
-  public Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>>
-      call(Iterator<WindowedValue<I>> iter) throws Exception {
-    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
-    mFunction.startBundle(ctxt);
-    ctxt.setup();
-    return ctxt.getOutputIterable(iter, mFunction);
-  }
-
-  private class ProcCtxt extends SparkProcessContext<I, O, Tuple2<TupleTag<?>, WindowedValue<?>>> {
-
-    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
-
-    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
-        BroadcastHelper<?>> sideInputs) {
-      super(fn, runtimeContext, sideInputs);
-    }
-
-    @Override
-    public synchronized void output(O o) {
-      outputs.put(mMainOutputTag, windowedValue.withValue(o));
-    }
-
-    @Override
-    public synchronized void output(WindowedValue<O> o) {
-      outputs.put(mMainOutputTag, o);
-    }
-
-    @Override
-    public synchronized <T> void sideOutput(TupleTag<T> tag, T t) {
-      outputs.put(tag, windowedValue.withValue(t));
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
-      outputs.put(tupleTag, WindowedValue.of(t, instant,
-          windowedValue.getWindows(), windowedValue.getPane()));
-    }
-
-    @Override
-    protected void clearOutput() {
-      outputs.clear();
-    }
-
-    @Override
-    protected Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> getOutputIterator() {
-      return Iterators.transform(outputs.entries().iterator(),
-          new Function<Map.Entry<TupleTag<?>, WindowedValue<?>>,
-              Tuple2<TupleTag<?>, WindowedValue<?>>>() {
-        @Override
-        public Tuple2<TupleTag<?>, WindowedValue<?>> apply(Map.Entry<TupleTag<?>,
-            WindowedValue<?>> input) {
-          return new Tuple2<TupleTag<?>, WindowedValue<?>>(input.getKey(), input.getValue());
-        }
-      });
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
deleted file mode 100644
index 4393a75..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkContextFactory.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.serializer.KryoSerializer;
-
-final class SparkContextFactory {
-
-  /**
-   * If the property {@code dataflow.spark.test.reuseSparkContext} is set to
-   * {@code true} then the Spark context will be reused for dataflow pipelines.
-   * This property should only be enabled for tests.
-   */
-  static final String TEST_REUSE_SPARK_CONTEXT =
-      "dataflow.spark.test.reuseSparkContext";
-  private static JavaSparkContext sparkContext;
-  private static String sparkMaster;
-
-  private SparkContextFactory() {
-  }
-
-  static synchronized JavaSparkContext getSparkContext(String master, String appName) {
-    if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
-      if (sparkContext == null) {
-        sparkContext = createSparkContext(master, appName);
-        sparkMaster = master;
-      } else if (!master.equals(sparkMaster)) {
-        throw new IllegalArgumentException(String.format("Cannot reuse spark context " +
-                "with different spark master URL. Existing: %s, requested: %s.",
-            sparkMaster, master));
-      }
-      return sparkContext;
-    } else {
-      return createSparkContext(master, appName);
-    }
-  }
-
-  static synchronized void stopSparkContext(JavaSparkContext context) {
-    if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
-      context.stop();
-    }
-  }
-
-  private static JavaSparkContext createSparkContext(String master, String appName) {
-    SparkConf conf = new SparkConf();
-    conf.setMaster(master);
-    conf.setAppName(appName);
-    conf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
-    return new JavaSparkContext(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
deleted file mode 100644
index becf15a..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineEvaluator.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-/**
- * Pipeline {@link SparkPipelineRunner.Evaluator} for Spark.
- */
-public final class SparkPipelineEvaluator extends SparkPipelineRunner.Evaluator {
-
-  private final EvaluationContext ctxt;
-
-  public SparkPipelineEvaluator(EvaluationContext ctxt, SparkPipelineTranslator translator) {
-    super(translator);
-    this.ctxt = ctxt;
-  }
-
-  @Override
-  protected <PT extends PTransform<? super PInput, POutput>> void doVisitTransform(TransformTreeNode
-      node) {
-    @SuppressWarnings("unchecked")
-    PT transform = (PT) node.getTransform();
-    @SuppressWarnings("unchecked")
-    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
-    @SuppressWarnings("unchecked") TransformEvaluator<PT> evaluator =
-        (TransformEvaluator<PT>) translator.translate(transformClass);
-    LOG.info("Evaluating {}", transform);
-    AppliedPTransform<PInput, POutput, PT> appliedTransform =
-        AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform);
-    ctxt.setCurrentTransform(appliedTransform);
-    evaluator.evaluate(transform, ctxt);
-    ctxt.setCurrentTransform(null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
deleted file mode 100644
index 9bff013..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-
-public final class SparkPipelineOptionsFactory {
-  private SparkPipelineOptionsFactory() {
-  }
-
-  public static SparkPipelineOptions create() {
-    return PipelineOptionsFactory.as(SparkPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
deleted file mode 100644
index c68af64..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptionsRegistrar.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
-  @Override
-  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
index b1a402f..d5e4186 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -28,10 +28,10 @@ import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.cloud.dataflow.sdk.values.PValue;
 
-import org.apache.beam.runners.spark.streaming.SparkStreamingPipelineOptions;
-import org.apache.beam.runners.spark.streaming.StreamingEvaluationContext;
-import org.apache.beam.runners.spark.streaming.StreamingTransformTranslator;
-import org.apache.beam.runners.spark.streaming.StreamingWindowPipelineDetector;
+import org.apache.beam.runners.spark.translation.*;
+import org.apache.beam.runners.spark.translation.streaming.StreamingEvaluationContext;
+import org.apache.beam.runners.spark.translation.streaming.StreamingTransformTranslator;
+import org.apache.beam.runners.spark.translation.streaming.StreamingWindowPipelineDetector;
 
 import org.apache.spark.SparkException;
 import org.apache.spark.api.java.JavaSparkContext;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
deleted file mode 100644
index 7861685..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunnerRegistrar.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkPipelineRunnerRegistrar implements PipelineRunnerRegistrar {
-  @Override
-  public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-    return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
deleted file mode 100644
index 2e38a07..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineTranslator.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-/**
- * Translator to support translation between Dataflow transformations and Spark transformations.
- */
-public interface SparkPipelineTranslator {
-
-  boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz);
-
-  <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
deleted file mode 100644
index b3a720d..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkProcessContext.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterables;
-
-import org.apache.beam.runners.spark.util.BroadcastHelper;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class SparkProcessContext<I, O, V> extends DoFn<I, O>.ProcessContext {
-
-  private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class);
-
-  private final DoFn<I, O> fn;
-  private final SparkRuntimeContext mRuntimeContext;
-  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
-
-  protected WindowedValue<I> windowedValue;
-
-  SparkProcessContext(DoFn<I, O> fn,
-      SparkRuntimeContext runtime,
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
-    fn.super();
-    this.fn = fn;
-    this.mRuntimeContext = runtime;
-    this.mSideInputs = sideInputs;
-  }
-
-  void setup() {
-    setupDelegateAggregators();
-  }
-
-  @Override
-  public PipelineOptions getPipelineOptions() {
-    return mRuntimeContext.getPipelineOptions();
-  }
-
-  @Override
-  public <T> T sideInput(PCollectionView<T> view) {
-    @SuppressWarnings("unchecked")
-    BroadcastHelper<Iterable<WindowedValue<?>>> broadcastHelper =
-        (BroadcastHelper<Iterable<WindowedValue<?>>>) mSideInputs.get(view.getTagInternal());
-    Iterable<WindowedValue<?>> contents = broadcastHelper.getValue();
-    return view.fromIterableInternal(contents);
-  }
-
-  @Override
-  public abstract void output(O output);
-
-  public abstract void output(WindowedValue<O> output);
-
-  @Override
-  public <T> void sideOutput(TupleTag<T> tupleTag, T t) {
-    String message = "sideOutput is an unsupported operation for doFunctions, use a " +
-        "MultiDoFunction instead.";
-    LOG.warn(message);
-    throw new UnsupportedOperationException(message);
-  }
-
-  @Override
-  public <T> void sideOutputWithTimestamp(TupleTag<T> tupleTag, T t, Instant instant) {
-    String message =
-        "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " +
-            "MultiDoFunction instead.";
-    LOG.warn(message);
-    throw new UnsupportedOperationException(message);
-  }
-
-  @Override
-  public <AI, AO> Aggregator<AI, AO> createAggregatorInternal(
-      String named,
-      Combine.CombineFn<AI, ?, AO> combineFn) {
-    return mRuntimeContext.createAggregator(named, combineFn);
-  }
-
-  @Override
-  public I element() {
-    return windowedValue.getValue();
-  }
-
-  @Override
-  public void outputWithTimestamp(O output, Instant timestamp) {
-    output(WindowedValue.of(output, timestamp,
-        windowedValue.getWindows(), windowedValue.getPane()));
-  }
-
-  @Override
-  public Instant timestamp() {
-    return windowedValue.getTimestamp();
-  }
-
-  @Override
-  public BoundedWindow window() {
-    if (!(fn instanceof DoFn.RequiresWindowAccess)) {
-      throw new UnsupportedOperationException(
-          "window() is only available in the context of a DoFn marked as RequiresWindow.");
-    }
-    return Iterables.getOnlyElement(windowedValue.getWindows());
-  }
-
-  @Override
-  public PaneInfo pane() {
-    return windowedValue.getPane();
-  }
-
-  @Override
-  public WindowingInternals<I, O> windowingInternals() {
-    return new WindowingInternals<I, O>() {
-
-      @Override
-      public Collection<? extends BoundedWindow> windows() {
-        return windowedValue.getWindows();
-      }
-
-      @Override
-      public void outputWindowedValue(O output, Instant timestamp, Collection<?
-          extends BoundedWindow> windows, PaneInfo paneInfo) {
-        output(WindowedValue.of(output, timestamp, windows, paneInfo));
-      }
-
-      @Override
-      public StateInternals stateInternals() {
-        //TODO: implement state internals.
-        // This is a temporary placeholder to get the TfIdfTest
-        // working for the initial Beam code drop.
-        return InMemoryStateInternals.forKey("DUMMY");
-      }
-
-      @Override
-      public TimerInternals timerInternals() {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#timerInternals() is not yet supported.");
-      }
-
-      @Override
-      public PaneInfo pane() {
-        return windowedValue.getPane();
-      }
-
-      @Override
-      public <T> void writePCollectionViewData(TupleTag<?> tag,
-          Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#writePCollectionViewData() is not yet supported.");
-      }
-
-      @Override
-      public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-        throw new UnsupportedOperationException(
-            "WindowingInternals#sideInput() is not yet supported.");
-      }
-    };
-  }
-
-  protected abstract void clearOutput();
-  protected abstract Iterator<V> getOutputIterator();
-
-  protected Iterable<V> getOutputIterable(final Iterator<WindowedValue<I>> iter,
-      final DoFn<I, O> doFn) {
-    return new Iterable<V>() {
-      @Override
-      public Iterator<V> iterator() {
-        return new ProcCtxtIterator(iter, doFn);
-      }
-    };
-  }
-
-  private class ProcCtxtIterator extends AbstractIterator<V> {
-
-    private final Iterator<WindowedValue<I>> inputIterator;
-    private final DoFn<I, O> doFn;
-    private Iterator<V> outputIterator;
-    private boolean calledFinish;
-
-    ProcCtxtIterator(Iterator<WindowedValue<I>> iterator, DoFn<I, O> doFn) {
-      this.inputIterator = iterator;
-      this.doFn = doFn;
-      this.outputIterator = getOutputIterator();
-    }
-
-    @Override
-    protected V computeNext() {
-      // Process each element from the (input) iterator, which produces, zero, one or more
-      // output elements (of type V) in the output iterator. Note that the output
-      // collection (and iterator) is reset between each call to processElement, so the
-      // collection only holds the output values for each call to processElement, rather
-      // than for the whole partition (which would use too much memory).
-      while (true) {
-        if (outputIterator.hasNext()) {
-          return outputIterator.next();
-        } else if (inputIterator.hasNext()) {
-          clearOutput();
-          windowedValue = inputIterator.next();
-          try {
-            doFn.processElement(SparkProcessContext.this);
-          } catch (Exception e) {
-            throw new SparkProcessException(e);
-          }
-          outputIterator = getOutputIterator();
-        } else {
-          // no more input to consume, but finishBundle can produce more output
-          if (!calledFinish) {
-            clearOutput();
-            try {
-              calledFinish = true;
-              doFn.finishBundle(SparkProcessContext.this);
-            } catch (Exception e) {
-              throw new SparkProcessException(e);
-            }
-            outputIterator = getOutputIterator();
-            continue; // try to consume outputIterator from start of loop
-          }
-          return endOfData();
-        }
-      }
-    }
-  }
-
-  static class SparkProcessException extends RuntimeException {
-    SparkProcessException(Throwable t) {
-      super(t);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
deleted file mode 100644
index f0f9974..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRuntimeContext.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.collect.ImmutableList;
-
-import org.apache.beam.runners.spark.aggregators.AggAccumParam;
-import org.apache.beam.runners.spark.aggregators.NamedAggregators;
-
-import org.apache.spark.Accumulator;
-import org.apache.spark.api.java.JavaSparkContext;
-
-
-/**
- * The SparkRuntimeContext allows us to define useful features on the client side before our
- * data flow program is launched.
- */
-public class SparkRuntimeContext implements Serializable {
-  /**
-   * An accumulator that is a map from names to aggregators.
-   */
-  private final Accumulator<NamedAggregators> accum;
-
-  private final String serializedPipelineOptions;
-
-  /**
-   * Map fo names to dataflow aggregators.
-   */
-  private final Map<String, Aggregator<?, ?>> aggregators = new HashMap<>();
-  private transient CoderRegistry coderRegistry;
-
-  SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) {
-    this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam());
-    this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions());
-  }
-
-  private static String serializePipelineOptions(PipelineOptions pipelineOptions) {
-    try {
-      return new ObjectMapper().writeValueAsString(pipelineOptions);
-    } catch (JsonProcessingException e) {
-      throw new IllegalStateException("Failed to serialize the pipeline options.", e);
-    }
-  }
-
-  private static PipelineOptions deserializePipelineOptions(String serializedPipelineOptions) {
-    try {
-      return new ObjectMapper().readValue(serializedPipelineOptions, PipelineOptions.class);
-    } catch (IOException e) {
-      throw new IllegalStateException("Failed to deserialize the pipeline options.", e);
-    }
-  }
-
-  /**
-   * Retrieves corresponding value of an aggregator.
-   *
-   * @param aggregatorName Name of the aggregator to retrieve the value of.
-   * @param typeClass      Type class of value to be retrieved.
-   * @param <T>            Type of object to be returned.
-   * @return The value of the aggregator.
-   */
-  public <T> T getAggregatorValue(String aggregatorName, Class<T> typeClass) {
-    return accum.value().getValue(aggregatorName, typeClass);
-  }
-
-  public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
-    @SuppressWarnings("unchecked")
-    Class<T> aggValueClass = (Class<T>) aggregator.getCombineFn().getOutputType().getRawType();
-    final T aggregatorValue = getAggregatorValue(aggregator.getName(), aggValueClass);
-    return new AggregatorValues<T>() {
-      @Override
-      public Collection<T> getValues() {
-        return ImmutableList.of(aggregatorValue);
-      }
-
-      @Override
-      public Map<String, T> getValuesAtSteps() {
-        throw new UnsupportedOperationException("getValuesAtSteps is not supported.");
-      }
-    };
-  }
-
-  public synchronized PipelineOptions getPipelineOptions() {
-    return deserializePipelineOptions(serializedPipelineOptions);
-  }
-
-  /**
-   * Creates and aggregator and associates it with the specified name.
-   *
-   * @param named     Name of aggregator.
-   * @param combineFn Combine function used in aggregation.
-   * @param <IN>      Type of inputs to aggregator.
-   * @param <INTER>   Intermediate data type
-   * @param <OUT>     Type of aggregator outputs.
-   * @return Specified aggregator
-   */
-  public synchronized <IN, INTER, OUT> Aggregator<IN, OUT> createAggregator(
-      String named,
-      Combine.CombineFn<? super IN, INTER, OUT> combineFn) {
-    @SuppressWarnings("unchecked")
-    Aggregator<IN, OUT> aggregator = (Aggregator<IN, OUT>) aggregators.get(named);
-    if (aggregator == null) {
-      @SuppressWarnings("unchecked")
-      NamedAggregators.CombineFunctionState<IN, INTER, OUT> state =
-          new NamedAggregators.CombineFunctionState<>(
-              (Combine.CombineFn<IN, INTER, OUT>) combineFn,
-              (Coder<IN>) getCoder(combineFn),
-              this);
-      accum.add(new NamedAggregators(named, state));
-      aggregator = new SparkAggregator<>(named, state);
-      aggregators.put(named, aggregator);
-    }
-    return aggregator;
-  }
-
-  public CoderRegistry getCoderRegistry() {
-    if (coderRegistry == null) {
-      coderRegistry = new CoderRegistry();
-      coderRegistry.registerStandardCoders();
-    }
-    return coderRegistry;
-  }
-
-  private Coder<?> getCoder(Combine.CombineFn<?, ?, ?> combiner) {
-    try {
-      if (combiner.getClass() == Sum.SumIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Sum.SumLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Sum.SumDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else if (combiner.getClass() == Min.MinIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Min.MinLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Min.MinDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else if (combiner.getClass() == Max.MaxIntegerFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class));
-      } else if (combiner.getClass() == Max.MaxLongFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Long.class));
-      } else if (combiner.getClass() == Max.MaxDoubleFn.class) {
-        return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Double.class));
-      } else {
-        throw new IllegalArgumentException("unsupported combiner in Aggregator: "
-            + combiner.getClass().getName());
-      }
-    } catch (CannotProvideCoderException e) {
-      throw new IllegalStateException("Could not determine default coder for combiner", e);
-    }
-  }
-
-  /**
-   * Initialize spark aggregators exactly once.
-   *
-   * @param <IN> Type of element fed in to aggregator.
-   */
-  private static class SparkAggregator<IN, OUT> implements Aggregator<IN, OUT>, Serializable {
-    private final String name;
-    private final NamedAggregators.State<IN, ?, OUT> state;
-
-    SparkAggregator(String name, NamedAggregators.State<IN, ?, OUT> state) {
-      this.name = name;
-      this.state = state;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public void addValue(IN elem) {
-      state.update(elem);
-    }
-
-    @Override
-    public Combine.CombineFn<IN, ?, OUT> getCombineFn() {
-      return state.getCombineFn();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
new file mode 100644
index 0000000..be40313
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+
+/**
+ * Options used to configure Spark streaming.
+ */
+public interface SparkStreamingPipelineOptions extends SparkPipelineOptions {
+  @Description("Timeout to wait (in msec) for the streaming execution so stop, -1 runs until " +
+          "execution is stopped")
+  @Default.Long(-1)
+  Long getTimeout();
+
+  void setTimeout(Long batchInterval);
+
+  @Override
+  @Default.Boolean(true)
+  boolean isStreaming();
+
+  @Override
+  @Default.String("spark streaming dataflow pipeline job")
+  String getAppName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eb0341d4/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
deleted file mode 100644
index 4b4f81f..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TransformEvaluator.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import java.io.Serializable;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-public interface TransformEvaluator<PT extends PTransform<?, ?>> extends Serializable {
-  void evaluate(PT transform, EvaluationContext context);
-}


[20/23] incubator-beam git commit: [BEAM-11] Replaced license headers to ASF license

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
index af831c6..c3588c9 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.util;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
index 7679b9c..b1254d4 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.util;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
index 49e68f5..35a634a 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
index aaa0dac..a4c5eb7 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
index 3af0a57..905e243 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
index 2aa1e6a..1ec3d75 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
index cd7bc11..e0fe47d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
index d090cf6..c89090d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
index 35bbf91..ae1eed7 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
index 44d8e0f..bdc048c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SideEffectsTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
index f930855..c7dc400 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
index 3fc3ecc..23416d7 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TestSparkPipelineOptionsFactory.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
index 0a36c9e..51ce3a3 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
index f759fe9..ed58c77 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TransformTranslatorTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
index eb88542..77409a0 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/WindowedWordCountTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
index ad7256c..8ba64ec 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
index 73dd2d3..b8e2697 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
index 39525b2..5609e88 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
index abe1119..9e5fdbc 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
index b1d35d5..90053ac 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
 package org.apache.beam.runners.spark.io.hadoop;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
index 828b26e..a7b9f28 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/FlattenStreamingTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
index ff1e11c..05340d6 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/KafkaStreamingTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
index 9a0609d..16b145a 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/SimpleStreamingWordCountTest.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
index 19759d7..367a062 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/DataflowAssertStreaming.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ef36411/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
index 333453a..8273684 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/streaming/utils/EmbeddedKafkaCluster.java
@@ -1,16 +1,19 @@
 /*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ * 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
  *
- * Cloudera, Inc. 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
  *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * This software 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.beam.runners.spark.streaming.utils;
 


[22/23] incubator-beam git commit: [BEAM-11] remove gitignore and travis.yml

Posted by am...@apache.org.
[BEAM-11] remove gitignore and travis.yml


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

Branch: refs/heads/master
Commit: 3980f44e96ffee61f9939a1edf62fa11502eb870
Parents: 6ef3641
Author: Sela <an...@paypal.com>
Authored: Mon Mar 14 20:48:15 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:40:18 2016 +0200

----------------------------------------------------------------------
 runners/spark/.gitignore  | 10 ----------
 runners/spark/.travis.yml | 22 ----------------------
 2 files changed, 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3980f44e/runners/spark/.gitignore
----------------------------------------------------------------------
diff --git a/runners/spark/.gitignore b/runners/spark/.gitignore
deleted file mode 100644
index 0b4510c..0000000
--- a/runners/spark/.gitignore
+++ /dev/null
@@ -1,10 +0,0 @@
-.classpath
-.project
-.settings
-.cache
-target
-*.iml
-.idea
-gen
-.DS_Store
-dependency-reduced-pom.xml
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3980f44e/runners/spark/.travis.yml
----------------------------------------------------------------------
diff --git a/runners/spark/.travis.yml b/runners/spark/.travis.yml
deleted file mode 100644
index c4af8a6..0000000
--- a/runners/spark/.travis.yml
+++ /dev/null
@@ -1,22 +0,0 @@
-language: java
-sudo: false
-install: mvn ${JAVA} ${SPARK} -DskipTests=true -Dmaven.javadoc.skip=true -B -V install
-script: mvn ${JAVA} ${SPARK} ${JACOCO} -Dmaven.javadoc.skip=true -B verify
-matrix:
- include:
-   # Covers Java 7, Open JDK, Spark 1.3.x, and code coverage
-   - jdk: openjdk7
-     env: JACOCO=-Pjacoco
-   # Covers Spark 1.4.x
-   - jdk: openjdk7
-     env: SPARK=-Dspark.version=1.4.1
-   # Covers Spark 1.5.x
-   - jdk: openjdk7
-     env: SPARK=-Dspark.version=1.5.1
-   # Covers Java 8, Oracle JDK
-   - jdk: oraclejdk8
-     env: JAVA=-Djava.version=1.8
-cache:
-  directories:
-    - $HOME/.m2
-after_success: if [ -n "$JACOCO" ]; then bash <(curl -s https://codecov.io/bash); fi


[05/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java
deleted file mode 100644
index 6849389..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Sets;
-import com.google.common.io.Files;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class NumShardsTest {
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
-  private File outputDir;
-
-  @Rule
-  public final TemporaryFolder tmpDir = new TemporaryFolder();
-
-  @Before
-  public void setUp() throws IOException {
-    outputDir = tmpDir.newFolder("out");
-    outputDir.delete();
-  }
-
-  @Test
-  public void testText() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-    PCollection<String> output = inputWords.apply(new WordCount.CountWords())
-        .apply(MapElements.via(new WordCount.FormatAsTextFn()));
-    output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt"));
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-
-    int count = 0;
-    Set<String> expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2");
-    for (File f : tmpDir.getRoot().listFiles(new FileFilter() {
-      @Override public boolean accept(File pathname) {
-        return pathname.getName().matches("out-.*\\.txt");
-      }
-    })) {
-      count++;
-      for (String line : Files.readLines(f, Charsets.UTF_8)) {
-        assertTrue(line + " not found", expected.remove(line));
-      }
-    }
-    assertEquals(3, count);
-    assertTrue(expected.isEmpty());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java
deleted file mode 100644
index 21a839b..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.junit.Test;
-
-public class SerializationTest {
-
-  public static class StringHolder { // not serializable
-    private final String string;
-
-    public StringHolder(String string) {
-      this.string = string;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      StringHolder that = (StringHolder) o;
-      return string.equals(that.string);
-    }
-
-    @Override
-    public int hashCode() {
-      return string.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return string;
-    }
-  }
-
-  public static class StringHolderUtf8Coder extends AtomicCoder<StringHolder> {
-
-    private final StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of();
-
-    @Override
-    public void encode(StringHolder value, OutputStream outStream, Context context) throws IOException {
-      stringUtf8Coder.encode(value.toString(), outStream, context);
-    }
-
-    @Override
-    public StringHolder decode(InputStream inStream, Context context) throws IOException {
-      return new StringHolder(stringUtf8Coder.decode(inStream, context));
-    }
-
-    public static Coder<StringHolder> of() {
-      return new StringHolderUtf8Coder();
-    }
-  }
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<StringHolder> WORDS = Lists.transform(
-      Arrays.asList(WORDS_ARRAY), new Function<String, StringHolder>() {
-        @Override public StringHolder apply(String s) {
-          return new StringHolder(s);
-        }
-      });
-  private static final Set<StringHolder> EXPECTED_COUNT_SET =
-      ImmutableSet.copyOf(Lists.transform(
-          Arrays.asList("hi: 5", "there: 1", "sue: 2", "bob: 2"),
-          new Function<String, StringHolder>() {
-            @Override
-            public StringHolder apply(String s) {
-              return new StringHolder(s);
-            }
-          }));
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    PCollection<StringHolder> inputWords =
-        p.apply(Create.of(WORDS).withCoder(StringHolderUtf8Coder.of()));
-    PCollection<StringHolder> output = inputWords.apply(new CountWords());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-
-  /**
-   * A DoFn that tokenizes lines of text into individual words.
-   */
-  static class ExtractWordsFn extends DoFn<StringHolder, StringHolder> {
-    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      // Split the line into words.
-      String[] words = WORD_BOUNDARY.split(c.element().toString());
-
-      // Keep track of the number of lines without any words encountered while tokenizing.
-      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
-      if (words.length == 0) {
-        emptyLines.addValue(1L);
-      }
-
-      // Output each word encountered into the output PCollection.
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(new StringHolder(word));
-        }
-      }
-    }
-  }
-
-  /**
-   * A DoFn that converts a Word and Count into a printable string.
-   */
-  private static class FormatCountsFn extends DoFn<KV<StringHolder, Long>, StringHolder> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(new StringHolder(c.element().getKey() + ": " + c.element().getValue()));
-    }
-  }
-
-  private static class CountWords extends PTransform<PCollection<StringHolder>, PCollection<StringHolder>> {
-    @Override
-    public PCollection<StringHolder> apply(PCollection<StringHolder> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<StringHolder> words = lines.apply(
-          ParDo.of(new ExtractWordsFn()));
-
-      // Count the number of times each word occurs.
-      PCollection<KV<StringHolder, Long>> wordCounts =
-          words.apply(Count.<StringHolder>perElement());
-
-      // Format each word and count into a printable string.
-
-      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java
deleted file mode 100644
index 341c214..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import org.junit.Test;
-
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputDirectory;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputFilePrefix;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputFileTemplate;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardCount;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardNumber;
-import static org.junit.Assert.assertEquals;
-
-public class ShardNameBuilderTest {
-
-  @Test
-  public void testReplaceShardCount() {
-    assertEquals("", replaceShardCount("", 6));
-    assertEquals("-S-of-6", replaceShardCount("-S-of-N", 6));
-    assertEquals("-SS-of-06", replaceShardCount("-SS-of-NN", 6));
-    assertEquals("-S-of-60", replaceShardCount("-S-of-N", 60));
-    assertEquals("-SS-of-60", replaceShardCount("-SS-of-NN", 60));
-    assertEquals("/part-SSSSS", replaceShardCount("/part-SSSSS", 6));
-  }
-
-  @Test
-  public void testReplaceShardNumber() {
-    assertEquals("", replaceShardNumber("", 5));
-    assertEquals("-5-of-6", replaceShardNumber("-S-of-6", 5));
-    assertEquals("-05-of-06", replaceShardNumber("-SS-of-06", 5));
-    assertEquals("-59-of-60", replaceShardNumber("-S-of-60", 59));
-    assertEquals("-59-of-60", replaceShardNumber("-SS-of-60", 59));
-    assertEquals("/part-00005", replaceShardNumber("/part-SSSSS", 5));
-  }
-
-  @Test
-     public void testGetOutputDirectory() {
-    assertEquals("./", getOutputDirectory("foo", "-S-of-N"));
-    assertEquals("foo", getOutputDirectory("foo/bar", "-S-of-N"));
-    assertEquals("/foo", getOutputDirectory("/foo/bar", "-S-of-N"));
-    assertEquals("hdfs://foo/", getOutputDirectory("hdfs://foo/bar", "-S-of-N"));
-    assertEquals("foo/bar", getOutputDirectory("foo/bar", "/part-SSSSS"));
-    assertEquals("/foo/bar", getOutputDirectory("/foo/bar", "/part-SSSSS"));
-    assertEquals("hdfs://foo/bar", getOutputDirectory("hdfs://foo/bar", "/part-SSSSS"));
-  }
-
-  @Test
-  public void testGetOutputFilePrefix() {
-    assertEquals("foo", getOutputFilePrefix("foo", "-S-of-N"));
-    assertEquals("bar", getOutputFilePrefix("foo/bar", "-S-of-N"));
-    assertEquals("bar", getOutputFilePrefix("/foo/bar", "-S-of-N"));
-    assertEquals("bar", getOutputFilePrefix("hdfs://foo/bar", "-S-of-N"));
-    assertEquals("", getOutputFilePrefix("foo/bar", "/part-SSSSS"));
-    assertEquals("", getOutputFilePrefix("/foo/bar", "/part-SSSSS"));
-    assertEquals("", getOutputFilePrefix("hdfs://foo/bar", "/part-SSSSS"));
-  }
-
-  @Test
-  public void testGetOutputFileTemplate() {
-    assertEquals("-S-of-N", getOutputFileTemplate("foo", "-S-of-N"));
-    assertEquals("-S-of-N", getOutputFileTemplate("foo/bar", "-S-of-N"));
-    assertEquals("-S-of-N", getOutputFileTemplate("/foo/bar", "-S-of-N"));
-    assertEquals("-S-of-N", getOutputFileTemplate("hdfs://foo/bar", "-S-of-N"));
-    assertEquals("part-SSSSS", getOutputFileTemplate("foo/bar", "/part-SSSSS"));
-    assertEquals("part-SSSSS", getOutputFileTemplate("/foo/bar", "/part-SSSSS"));
-    assertEquals("part-SSSSS", getOutputFileTemplate("hdfs://foo/bar", "/part-SSSSS"));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java
deleted file mode 100644
index ce7acda..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import java.io.Serializable;
-import java.net.URI;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class SideEffectsTest implements Serializable {
-
-  static class UserException extends RuntimeException {
-  }
-
-  @Test
-  public void test() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
-    pipeline.apply(Create.of("a")).apply(ParDo.of(new DoFn<String, String>() {
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        throw new UserException();
-      }
-    }));
-
-    try {
-      pipeline.run();
-      fail("Run should thrown an exception");
-    } catch (RuntimeException e) {
-      assertNotNull(e.getCause());
-
-      // TODO: remove the version check (and the setup and teardown methods) when we no
-      // longer support Spark 1.3 or 1.4
-      String version = SparkContextFactory.getSparkContext(options.getSparkMaster(), options.getAppName()).version();
-      if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) {
-        assertTrue(e.getCause() instanceof UserException);
-      }
-    }
-  }
-
-  @Before
-  public void setup() {
-    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "true");
-  }
-
-  @After
-  public void teardown() {
-    System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "false");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java
deleted file mode 100644
index 1c2f7a9..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.junit.Test;
-
-public class SimpleWordCountTest {
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob",
-      "hi sue", "", "bob hi"};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-  private static final Set<String> EXPECTED_COUNT_SET =
-      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder
-        .of());
-    PCollection<String> output = inputWords.apply(new CountWords());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-
-  /**
-   * A DoFn that tokenizes lines of text into individual words.
-   */
-  static class ExtractWordsFn extends DoFn<String, String> {
-    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      // Split the line into words.
-      String[] words = WORD_BOUNDARY.split(c.element());
-
-      // Keep track of the number of lines without any words encountered while tokenizing.
-      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
-      if (words.length == 0) {
-        emptyLines.addValue(1L);
-      }
-
-      // Output each word encountered into the output PCollection.
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(word);
-        }
-      }
-    }
-  }
-
-  /**
-   * A DoFn that converts a Word and Count into a printable string.
-   */
-  private static class FormatCountsFn extends DoFn<KV<String, Long>, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element().getKey() + ": " + c.element().getValue());
-    }
-  }
-
-  public static class CountWords extends PTransform<PCollection<String>, PCollection<String>> {
-    @Override
-    public PCollection<String> apply(PCollection<String> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<String> words = lines.apply(
-          ParDo.of(new ExtractWordsFn()));
-
-      // Count the number of times each word occurs.
-      PCollection<KV<String, Long>> wordCounts =
-          words.apply(Count.<String>perElement());
-
-      // Format each word and count into a printable string.
-
-      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java
deleted file mode 100644
index 50cd0de..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestSparkPipelineOptionsFactory {
-  @Test
-  public void testDefaultCreateMethod() {
-    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
-    Assert.assertEquals("local[1]", actualOptions.getSparkMaster());
-  }
-
-  @Test
-  public void testSettingCustomOptions() {
-    SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create();
-    actualOptions.setSparkMaster("spark://207.184.161.138:7077");
-    Assert.assertEquals("spark://207.184.161.138:7077", actualOptions.getSparkMaster());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java
deleted file mode 100644
index 35ab26e..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.examples.complete.TfIdf;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import java.net.URI;
-import java.util.Arrays;
-import org.junit.Test;
-
-/**
- * A test based on {@code TfIdf} from the SDK.
- */
-public class TfIdfTest {
-
-  @Test
-  public void testTfIdf() throws Exception {
-    Pipeline pipeline = Pipeline.create(PipelineOptionsFactory.create());
-
-    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
-    PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
-        .apply(Create.of(
-            KV.of(new URI("x"), "a b c d"),
-            KV.of(new URI("y"), "a b c"),
-            KV.of(new URI("z"), "a m n")))
-        .apply(new TfIdf.ComputeTfIdf());
-
-    PCollection<String> words = wordToUriAndTfIdf
-        .apply(Keys.<String>create())
-        .apply(RemoveDuplicates.<String>create());
-
-    DataflowAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d"));
-
-    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
-    res.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java
deleted file mode 100644
index 73b3643..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.api.client.repackaged.com.google.common.base.Joiner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Charsets;
-import java.util.Collections;
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-
-/**
- * A test for the transforms registered in TransformTranslator.
- * Builds a regular Dataflow pipeline with each of the mapped
- * transforms, and makes sure that they work when the pipeline is
- * executed in Spark.
- */
-public class TransformTranslatorTest {
-
-  @Rule
-  public TestName name = new TestName();
-
-  private DirectPipelineRunner directRunner;
-  private SparkPipelineRunner sparkRunner;
-  private String testDataDirName;
-
-  @Before public void init() throws IOException {
-    sparkRunner = SparkPipelineRunner.create();
-    directRunner = DirectPipelineRunner.createForTest();
-    testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName())
-        + File.separator;
-    FileUtils.deleteDirectory(new File(testDataDirName));
-    new File(testDataDirName).mkdirs();
-  }
-
-  /**
-   * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline
-   * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
-   * transforms. Finally it makes sure that the results are the same for both runs.
-   */
-  @Test
-  public void testTextIOReadAndWriteTransforms() throws IOException {
-    String directOut = runPipeline("direct", directRunner);
-    String sparkOut = runPipeline("spark", sparkRunner);
-
-    List<String> directOutput =
-        Files.readAllLines(Paths.get(directOut + "-00000-of-00001"), Charsets.UTF_8);
-
-    List<String> sparkOutput =
-        Files.readAllLines(Paths.get(sparkOut + "-00000-of-00001"), Charsets.UTF_8);
-
-    // sort output to get a stable result (PCollections are not ordered)
-    Collections.sort(directOutput);
-    Collections.sort(sparkOutput);
-
-    Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray());
-  }
-
-  private String runPipeline(String name, PipelineRunner<?> runner) {
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out_" + name);
-    PCollection<String> lines =  p.apply(TextIO.Read.from("src/test/resources/test_text.txt"));
-    lines.apply(TextIO.Write.to(outFile));
-    runner.run(p);
-    return outFile;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java
deleted file mode 100644
index c16878e..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
-import java.util.Arrays;
-import java.util.List;
-import org.joda.time.Duration;
-import org.junit.Test;
-
-public class WindowedWordCountTest {
-  private static final String[] WORDS_ARRAY = {
-          "hi there", "hi", "hi sue bob",
-          "hi sue", "", "bob hi"};
-  private static final Long[] TIMESTAMPS_ARRAY = {
-          60000L, 60000L, 60000L,
-          120000L, 120000L, 120000L};
-  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-  private static final List<Long> TIMESTAMPS = Arrays.asList(TIMESTAMPS_ARRAY);
-  private static final List<String> EXPECTED_COUNT_SET =
-          ImmutableList.of("hi: 3", "there: 1", "sue: 1", "bob: 1",
-                  "hi: 2", "sue: 1", "bob: 1");
-
-  @Test
-  public void testRun() throws Exception {
-    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
-    options.setRunner(SparkPipelineRunner.class);
-    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
-    PCollection<String> inputWords = p.apply(Create.timestamped(WORDS, TIMESTAMPS))
-            .setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedWords = inputWords
-            .apply(Window.<String>into(FixedWindows.of(Duration.standardMinutes(1))));
-
-    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
-
-    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create().run(p);
-    res.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.java
deleted file mode 100644
index 0872834..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-
-import com.cloudera.dataflow.io.CreateStream;
-import com.cloudera.dataflow.spark.EvaluationResult;
-import com.cloudera.dataflow.spark.SparkPipelineRunner;
-import com.cloudera.dataflow.spark.streaming.utils.DataflowAssertStreaming;
-
-import org.joda.time.Duration;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Test Flatten (union) implementation for streaming.
- */
-public class FlattenStreamingTest {
-
-  private static final String[] WORDS_ARRAY_1 = {
-      "one", "two", "three", "four"};
-  private static final List<Iterable<String>> WORDS_QUEUE_1 =
-      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_1));
-  private static final String[] WORDS_ARRAY_2 = {
-          "five", "six", "seven", "eight"};
-  private static final List<Iterable<String>> WORDS_QUEUE_2 =
-          Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY_2));
-  private static final String[] EXPECTED_UNION = {
-          "one", "two", "three", "four", "five", "six", "seven", "eight"};
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @Test
-  public void testRun() throws Exception {
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<String> w1 =
-            p.apply(CreateStream.fromQueue(WORDS_QUEUE_1)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedW1 =
-            w1.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-    PCollection<String> w2 =
-            p.apply(CreateStream.fromQueue(WORDS_QUEUE_2)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedW2 =
-            w2.apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-    PCollectionList<String> list = PCollectionList.of(windowedW1).and(windowedW2);
-    PCollection<String> union = list.apply(Flatten.<String>pCollections());
-
-    DataflowAssert.thatIterable(union.apply(View.<String>asIterable()))
-            .containsInAnyOrder(EXPECTED_UNION);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/KafkaStreamingTest.java
deleted file mode 100644
index f68aea8..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/KafkaStreamingTest.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.repackaged.com.google.common.collect.ImmutableMap;
-import com.google.cloud.dataflow.sdk.repackaged.com.google.common.collect.ImmutableSet;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import com.cloudera.dataflow.io.KafkaIO;
-import com.cloudera.dataflow.spark.EvaluationResult;
-import com.cloudera.dataflow.spark.SparkPipelineRunner;
-import com.cloudera.dataflow.spark.streaming.utils.DataflowAssertStreaming;
-import com.cloudera.dataflow.spark.streaming.utils.EmbeddedKafkaCluster;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.joda.time.Duration;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import kafka.serializer.StringDecoder;
-
-/**
- * Test Kafka as input.
- */
-public class KafkaStreamingTest {
-  private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
-          new EmbeddedKafkaCluster.EmbeddedZookeeper(17001);
-  private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER =
-          new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(),
-                  new Properties(), Collections.singletonList(6667));
-  private static final String TOPIC = "kafka_dataflow_test_topic";
-  private static final Map<String, String> KAFKA_MESSAGES = ImmutableMap.of(
-          "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
-  );
-  private static final Set<String> EXPECTED = ImmutableSet.of(
-          "k1,v1", "k2,v2", "k3,v3", "k4,v4"
-  );
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @BeforeClass
-  public static void init() throws IOException {
-    EMBEDDED_ZOOKEEPER.startup();
-    EMBEDDED_KAFKA_CLUSTER.startup();
-
-    // write to Kafka
-    Properties producerProps = new Properties();
-    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
-    producerProps.put("request.required.acks", 1);
-    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
-    Serializer<String> stringSerializer = new StringSerializer();
-    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
-            new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
-      for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
-        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
-      }
-    }
-  }
-
-  @Test
-  public void testRun() throws Exception {
-    // test read from Kafka
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    Map<String, String> kafkaParams = ImmutableMap.of(
-            "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
-            "auto.offset.reset", "smallest"
-    );
-
-    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
-        StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
-        kafkaParams));
-    PCollection<KV<String, String>> windowedWords = kafkaInput
-        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
-
-    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn()));
-
-    DataflowAssert.thatIterable(formattedKV.apply(View.<String>asIterable()))
-        .containsInAnyOrder(EXPECTED);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    EMBEDDED_KAFKA_CLUSTER.shutdown();
-    EMBEDDED_ZOOKEEPER.shutdown();
-  }
-
-  private static class FormatKVFn extends DoFn<KV<String, String>, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element().getKey() + "," + c.element().getValue());
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java
deleted file mode 100644
index e22e616..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-
-import com.cloudera.dataflow.io.CreateStream;
-import com.cloudera.dataflow.spark.EvaluationResult;
-import com.cloudera.dataflow.spark.SimpleWordCountTest;
-import com.cloudera.dataflow.spark.SparkPipelineRunner;
-import com.cloudera.dataflow.spark.streaming.utils.DataflowAssertStreaming;
-
-import org.joda.time.Duration;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class SimpleStreamingWordCountTest {
-
-  private static final String[] WORDS_ARRAY = {
-      "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"};
-  private static final List<Iterable<String>> WORDS_QUEUE =
-      Collections.<Iterable<String>>singletonList(Arrays.asList(WORDS_ARRAY));
-  private static final Set<String> EXPECTED_COUNT_SET =
-      ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
-  private static final long TEST_TIMEOUT_MSEC = 1000L;
-
-  @Test
-  public void testRun() throws Exception {
-    SparkStreamingPipelineOptions options = SparkStreamingPipelineOptionsFactory.create();
-    options.setAppName(this.getClass().getSimpleName());
-    options.setRunner(SparkPipelineRunner.class);
-    options.setTimeout(TEST_TIMEOUT_MSEC);// run for one interval
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<String> inputWords =
-        p.apply(CreateStream.fromQueue(WORDS_QUEUE)).setCoder(StringUtf8Coder.of());
-    PCollection<String> windowedWords = inputWords
-        .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))));
-
-    PCollection<String> output = windowedWords.apply(new SimpleWordCountTest.CountWords());
-
-    DataflowAssert.thatIterable(output.apply(View.<String>asIterable()))
-        .containsInAnyOrder(EXPECTED_COUNT_SET);
-
-    EvaluationResult res = SparkPipelineRunner.create(options).run(p);
-    res.close();
-
-    DataflowAssertStreaming.assertNoFailures(res);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/DataflowAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/DataflowAssertStreaming.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/DataflowAssertStreaming.java
deleted file mode 100644
index c0c5976..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/DataflowAssertStreaming.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming.utils;
-
-import com.cloudera.dataflow.spark.EvaluationResult;
-
-import org.junit.Assert;
-
-/**
- * Since DataflowAssert doesn't propagate assert exceptions, use Aggregators to assert streaming
- * success/failure counters.
- */
-public final class DataflowAssertStreaming {
-  /**
-   * Copied aggregator names from {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert}
-   */
-  static final String SUCCESS_COUNTER = "DataflowAssertSuccess";
-  static final String FAILURE_COUNTER = "DataflowAssertFailure";
-
-  private DataflowAssertStreaming() {
-  }
-
-  public static void assertNoFailures(EvaluationResult res) {
-    int failures = res.getAggregatorValue(FAILURE_COUNTER, Integer.class);
-    Assert.assertEquals("Found " + failures + " failures, see the log for details", 0, failures);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java
deleted file mode 100644
index e75d729..0000000
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming.utils;
-
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * https://gist.github.com/fjavieralba/7930018
- */
-public class EmbeddedKafkaCluster {
-
-  private static final Logger LOG = LoggerFactory.getLogger(EmbeddedKafkaCluster.class);
-
-  private final List<Integer> ports;
-  private final String zkConnection;
-  private final Properties baseProperties;
-
-  private final String brokerList;
-
-  private final List<KafkaServer> brokers;
-  private final List<File> logDirs;
-
-  public EmbeddedKafkaCluster(String zkConnection) {
-    this(zkConnection, new Properties());
-  }
-
-  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties) {
-    this(zkConnection, baseProperties, Collections.singletonList(-1));
-  }
-
-  public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties, List<Integer> ports) {
-    this.zkConnection = zkConnection;
-    this.ports = resolvePorts(ports);
-    this.baseProperties = baseProperties;
-
-    this.brokers = new ArrayList<>();
-    this.logDirs = new ArrayList<>();
-
-    this.brokerList = constructBrokerList(this.ports);
-  }
-
-  private static List<Integer> resolvePorts(List<Integer> ports) {
-    List<Integer> resolvedPorts = new ArrayList<>();
-    for (Integer port : ports) {
-      resolvedPorts.add(resolvePort(port));
-    }
-    return resolvedPorts;
-  }
-
-  private static int resolvePort(int port) {
-    if (port == -1) {
-      return TestUtils.getAvailablePort();
-    }
-    return port;
-  }
-
-  private static String constructBrokerList(List<Integer> ports) {
-    StringBuilder sb = new StringBuilder();
-    for (Integer port : ports) {
-      if (sb.length() > 0) {
-        sb.append(",");
-      }
-      sb.append("localhost:").append(port);
-    }
-    return sb.toString();
-  }
-
-  public void startup() {
-    for (int i = 0; i < ports.size(); i++) {
-      Integer port = ports.get(i);
-      File logDir = TestUtils.constructTempDir("kafka-local");
-
-      Properties properties = new Properties();
-      properties.putAll(baseProperties);
-      properties.setProperty("zookeeper.connect", zkConnection);
-      properties.setProperty("broker.id", String.valueOf(i + 1));
-      properties.setProperty("host.name", "localhost");
-      properties.setProperty("port", Integer.toString(port));
-      properties.setProperty("log.dir", logDir.getAbsolutePath());
-      properties.setProperty("log.flush.interval.messages", String.valueOf(1));
-
-      KafkaServer broker = startBroker(properties);
-
-      brokers.add(broker);
-      logDirs.add(logDir);
-    }
-  }
-
-
-  private static KafkaServer startBroker(Properties props) {
-    KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
-    server.startup();
-    return server;
-  }
-
-  public Properties getProps() {
-    Properties props = new Properties();
-    props.putAll(baseProperties);
-    props.put("metadata.broker.list", brokerList);
-    props.put("zookeeper.connect", zkConnection);
-    return props;
-  }
-
-  public String getBrokerList() {
-    return brokerList;
-  }
-
-  public List<Integer> getPorts() {
-    return ports;
-  }
-
-  public String getZkConnection() {
-    return zkConnection;
-  }
-
-  public void shutdown() {
-    for (KafkaServer broker : brokers) {
-      try {
-        broker.shutdown();
-      } catch (Exception e) {
-        LOG.warn("{}", e.getMessage(), e);
-      }
-    }
-    for (File logDir : logDirs) {
-      try {
-        TestUtils.deleteFile(logDir);
-      } catch (FileNotFoundException e) {
-        LOG.warn("{}", e.getMessage(), e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "EmbeddedKafkaCluster{" + "brokerList='" + brokerList + "'}";
-  }
-
-  public static class EmbeddedZookeeper {
-    private int port = -1;
-    private int tickTime = 500;
-
-    private ServerCnxnFactory factory;
-    private File snapshotDir;
-    private File logDir;
-
-    public EmbeddedZookeeper() {
-      this(-1);
-    }
-
-    public EmbeddedZookeeper(int port) {
-      this(port, 500);
-    }
-
-    public EmbeddedZookeeper(int port, int tickTime) {
-      this.port = resolvePort(port);
-      this.tickTime = tickTime;
-    }
-
-    private static int resolvePort(int port) {
-      if (port == -1) {
-        return TestUtils.getAvailablePort();
-      }
-      return port;
-    }
-
-    public void startup() throws IOException {
-      if (this.port == -1) {
-        this.port = TestUtils.getAvailablePort();
-      }
-      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
-              1024);
-      this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
-      this.logDir = TestUtils.constructTempDir("embedded-zk/log");
-
-      try {
-        factory.startup(new ZooKeeperServer(snapshotDir, logDir, tickTime));
-      } catch (InterruptedException e) {
-        throw new IOException(e);
-      }
-    }
-
-
-    public void shutdown() {
-      factory.shutdown();
-      try {
-        TestUtils.deleteFile(snapshotDir);
-      } catch (FileNotFoundException e) {
-        // ignore
-      }
-      try {
-        TestUtils.deleteFile(logDir);
-      } catch (FileNotFoundException e) {
-        // ignore
-      }
-    }
-
-    public String getConnection() {
-      return "localhost:" + port;
-    }
-
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    public void setTickTime(int tickTime) {
-      this.tickTime = tickTime;
-    }
-
-    public int getPort() {
-      return port;
-    }
-
-    public int getTickTime() {
-      return tickTime;
-    }
-
-    @Override
-    public String toString() {
-      return "EmbeddedZookeeper{" + "connection=" + getConnection() + "}";
-    }
-  }
-
-  static class SystemTime implements Time {
-    @Override
-    public long milliseconds() {
-      return System.currentTimeMillis();
-    }
-
-    @Override
-    public long nanoseconds() {
-      return System.nanoTime();
-    }
-
-    @Override
-    public void sleep(long ms) {
-      try {
-        Thread.sleep(ms);
-      } catch (InterruptedException e) {
-        // Ignore
-      }
-    }
-  }
-
-  static final class TestUtils {
-    private static final Random RANDOM = new Random();
-
-    private TestUtils() {
-    }
-
-    static File constructTempDir(String dirPrefix) {
-      File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt
-              (10000000));
-      if (!file.mkdirs()) {
-        throw new RuntimeException("could not create temp directory: " + file.getAbsolutePath());
-      }
-      file.deleteOnExit();
-      return file;
-    }
-
-    static int getAvailablePort() {
-      try {
-        try (ServerSocket socket = new ServerSocket(0)) {
-          return socket.getLocalPort();
-        }
-      } catch (IOException e) {
-        throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e);
-      }
-    }
-
-    static boolean deleteFile(File path) throws FileNotFoundException {
-      if (!path.exists()) {
-        throw new FileNotFoundException(path.getAbsolutePath());
-      }
-      boolean ret = true;
-      if (path.isDirectory()) {
-        for (File f : path.listFiles()) {
-          ret = ret && deleteFile(f);
-        }
-      }
-      return ret && path.delete();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
new file mode 100644
index 0000000..e36babe
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombineGloballyTest.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class CombineGloballyTest {
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+    PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+    PCollection<String> output = inputWords.apply(Combine.globally(new WordMerger()));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    assertEquals("hi there,hi,hi sue bob,hi sue,,bob hi", Iterables.getOnlyElement(res.get(output)));
+    res.close();
+  }
+
+  public static class WordMerger extends Combine.CombineFn<String, StringBuilder, String> {
+
+    @Override
+    public StringBuilder createAccumulator() {
+      // return null to differentiate from an empty string
+      return null;
+    }
+
+    @Override
+    public StringBuilder addInput(StringBuilder accumulator, String input) {
+      return combine(accumulator, input);
+    }
+
+    @Override
+    public StringBuilder mergeAccumulators(Iterable<StringBuilder> accumulators) {
+      StringBuilder sb = new StringBuilder();
+      for (StringBuilder accum : accumulators) {
+        if (accum != null) {
+          sb.append(accum);
+        }
+      }
+      return sb;
+    }
+
+    @Override
+    public String extractOutput(StringBuilder accumulator) {
+      return accumulator.toString();
+    }
+
+    private static StringBuilder combine(StringBuilder accum, String datum) {
+      if (accum == null) {
+        return new StringBuilder(datum);
+      } else {
+        accum.append(",").append(datum);
+        return accum;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
new file mode 100644
index 0000000..aaa0dac
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CombinePerKeyTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CombinePerKeyTest {
+
+    private static final List<String> WORDS =
+        ImmutableList.of("the", "quick", "brown", "fox", "jumped", "over", "the", "lazy", "dog");
+    @Test
+    public void testRun() {
+        Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+        PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+        PCollection<KV<String, Long>> cnts = inputWords.apply(new SumPerKey<String>());
+        EvaluationResult res = SparkPipelineRunner.create().run(p);
+        Map<String, Long> actualCnts = new HashMap<>();
+        for (KV<String, Long> kv : res.get(cnts)) {
+            actualCnts.put(kv.getKey(), kv.getValue());
+        }
+        res.close();
+        Assert.assertEquals(8, actualCnts.size());
+        Assert.assertEquals(Long.valueOf(2L), actualCnts.get("the"));
+    }
+
+    private static class SumPerKey<T> extends PTransform<PCollection<T>, PCollection<KV<T, Long>>> {
+      @Override
+      public PCollection<KV<T, Long>> apply(PCollection<T> pcol) {
+          PCollection<KV<T, Long>> withLongs = pcol.apply(ParDo.of(new DoFn<T, KV<T, Long>>() {
+              @Override
+              public void processElement(ProcessContext processContext) throws Exception {
+                  processContext.output(KV.of(processContext.element(), 1L));
+              }
+          })).setCoder(KvCoder.of(pcol.getCoder(), VarLongCoder.of()));
+          return withLongs.apply(Sum.<T>longsPerKey());
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
new file mode 100644
index 0000000..3af0a57
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A test based on {@code DeDupExample} from the SDK.
+ */
+public class DeDupTest {
+
+  private static final String[] LINES_ARRAY = {
+      "hi there", "hello", "hi there",
+      "hi", "hello"};
+  private static final List<String> LINES = Arrays.asList(LINES_ARRAY);
+  private static final Set<String> EXPECTED_SET =
+      ImmutableSet.of("hi there", "hi", "hello");
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(options);
+    PCollection<String> input = p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of());
+    PCollection<String> output = input.apply(RemoveDuplicates.<String>create());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
new file mode 100644
index 0000000..2aa1e6a
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DoFnOutputTest.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.junit.Test;
+
+import java.io.Serializable;
+
+public class DoFnOutputTest implements Serializable {
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    PCollection<String> strings = pipeline.apply(Create.of("a"));
+    // Test that values written from startBundle() and finishBundle() are written to
+    // the output
+    PCollection<String> output = strings.apply(ParDo.of(new DoFn<String, String>() {
+      @Override
+      public void startBundle(Context c) throws Exception {
+        c.output("start");
+      }
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(c.element());
+      }
+      @Override
+      public void finishBundle(Context c) throws Exception {
+        c.output("finish");
+      }
+    }));
+
+    DataflowAssert.that(output).containsInAnyOrder("start", "a", "finish");
+
+    EvaluationResult res = SparkPipelineRunner.create().run(pipeline);
+    res.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
new file mode 100644
index 0000000..cd7bc11
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class EmptyInputTest {
+
+  @Test
+  public void test() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+    List<String> empty = Collections.emptyList();
+    PCollection<String> inputWords = p.apply(Create.of(empty)).setCoder(StringUtf8Coder.of());
+    PCollection<String> output = inputWords.apply(Combine.globally(new ConcatWords()));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    assertEquals("", Iterables.getOnlyElement(res.get(output)));
+    res.close();
+  }
+
+  public static class ConcatWords implements SerializableFunction<Iterable<String>, String> {
+    @Override
+    public String apply(Iterable<String> input) {
+      StringBuilder all = new StringBuilder();
+      for (String item : input) {
+        if (!item.isEmpty()) {
+          if (all.length() == 0) {
+            all.append(item);
+          } else {
+            all.append(",");
+            all.append(item);
+          }
+        }
+      }
+      return all.toString();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
new file mode 100644
index 0000000..d090cf6
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/MultiOutputWordCountTest.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.*;
+import com.google.common.collect.Iterables;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MultiOutputWordCountTest {
+
+  private static final TupleTag<String> upper = new TupleTag<>();
+  private static final TupleTag<String> lower = new TupleTag<>();
+  private static final TupleTag<KV<String, Long>> lowerCnts = new TupleTag<>();
+  private static final TupleTag<KV<String, Long>> upperCnts = new TupleTag<>();
+
+  @Test
+  public void testRun() throws Exception {
+    Pipeline p = Pipeline.create(PipelineOptionsFactory.create());
+    PCollection<String> regex = p.apply(Create.of("[^a-zA-Z']+"));
+    PCollection<String> w1 = p.apply(Create.of("Here are some words to count", "and some others"));
+    PCollection<String> w2 = p.apply(Create.of("Here are some more words", "and even more words"));
+    PCollectionList<String> list = PCollectionList.of(w1).and(w2);
+
+    PCollection<String> union = list.apply(Flatten.<String>pCollections());
+    PCollectionView<String> regexView = regex.apply(View.<String>asSingleton());
+    CountWords countWords = new CountWords(regexView);
+    PCollectionTuple luc = union.apply(countWords);
+    PCollection<Long> unique = luc.get(lowerCnts).apply(
+        ApproximateUnique.<KV<String, Long>>globally(16));
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    Iterable<KV<String, Long>> actualLower = res.get(luc.get(lowerCnts));
+    Assert.assertEquals("are", actualLower.iterator().next().getKey());
+    Iterable<KV<String, Long>> actualUpper = res.get(luc.get(upperCnts));
+    Assert.assertEquals("Here", actualUpper.iterator().next().getKey());
+    Iterable<Long> actualUniqCount = res.get(unique);
+    Assert.assertEquals(9, (long) actualUniqCount.iterator().next());
+    int actualTotalWords = res.getAggregatorValue("totalWords", Integer.class);
+    Assert.assertEquals(18, actualTotalWords);
+    int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class);
+    Assert.assertEquals(6, actualMaxWordLength);
+    AggregatorValues<Integer> aggregatorValues = res.getAggregatorValues(countWords
+        .getTotalWordsAggregator());
+    Assert.assertEquals(18, Iterables.getOnlyElement(aggregatorValues.getValues()).intValue());
+
+    res.close();
+  }
+
+  /**
+   * A DoFn that tokenizes lines of text into individual words.
+   */
+  static class ExtractWordsFn extends DoFn<String, String> {
+
+    private final Aggregator<Integer, Integer> totalWords = createAggregator("totalWords",
+        new Sum.SumIntegerFn());
+    private final Aggregator<Integer, Integer> maxWordLength = createAggregator("maxWordLength",
+        new Max.MaxIntegerFn());
+    private final PCollectionView<String> regex;
+
+    ExtractWordsFn(PCollectionView<String> regex) {
+      this.regex = regex;
+    }
+
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] words = c.element().split(c.sideInput(regex));
+      for (String word : words) {
+        totalWords.addValue(1);
+        if (!word.isEmpty()) {
+          maxWordLength.addValue(word.length());
+          if (Character.isLowerCase(word.charAt(0))) {
+            c.output(word);
+          } else {
+            c.sideOutput(upper, word);
+          }
+        }
+      }
+    }
+  }
+
+  public static class CountWords extends PTransform<PCollection<String>, PCollectionTuple> {
+
+    private final PCollectionView<String> regex;
+    private final ExtractWordsFn extractWordsFn;
+
+    public CountWords(PCollectionView<String> regex) {
+      this.regex = regex;
+      this.extractWordsFn = new ExtractWordsFn(regex);
+    }
+
+    @Override
+    public PCollectionTuple apply(PCollection<String> lines) {
+      // Convert lines of text into individual words.
+      PCollectionTuple lowerUpper = lines
+          .apply(ParDo.of(extractWordsFn)
+              .withSideInputs(regex)
+              .withOutputTags(lower, TupleTagList.of(upper)));
+      lowerUpper.get(lower).setCoder(StringUtf8Coder.of());
+      lowerUpper.get(upper).setCoder(StringUtf8Coder.of());
+      PCollection<KV<String, Long>> lowerCounts = lowerUpper.get(lower).apply(Count
+          .<String>perElement());
+      PCollection<KV<String, Long>> upperCounts = lowerUpper.get(upper).apply(Count
+          .<String>perElement());
+      return PCollectionTuple
+          .of(lowerCnts, lowerCounts)
+          .and(upperCnts, upperCounts);
+    }
+
+    Aggregator<Integer, Integer> getTotalWordsAggregator() {
+      return extractWordsFn.totalWords;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
new file mode 100644
index 0000000..35bbf91
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SerializationTest.java
@@ -0,0 +1,177 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class SerializationTest {
+
+  public static class StringHolder { // not serializable
+    private final String string;
+
+    public StringHolder(String string) {
+      this.string = string;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      StringHolder that = (StringHolder) o;
+      return string.equals(that.string);
+    }
+
+    @Override
+    public int hashCode() {
+      return string.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return string;
+    }
+  }
+
+  public static class StringHolderUtf8Coder extends AtomicCoder<StringHolder> {
+
+    private final StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of();
+
+    @Override
+    public void encode(StringHolder value, OutputStream outStream, Context context) throws IOException {
+      stringUtf8Coder.encode(value.toString(), outStream, context);
+    }
+
+    @Override
+    public StringHolder decode(InputStream inStream, Context context) throws IOException {
+      return new StringHolder(stringUtf8Coder.decode(inStream, context));
+    }
+
+    public static Coder<StringHolder> of() {
+      return new StringHolderUtf8Coder();
+    }
+  }
+
+  private static final String[] WORDS_ARRAY = {
+      "hi there", "hi", "hi sue bob",
+      "hi sue", "", "bob hi"};
+  private static final List<StringHolder> WORDS = Lists.transform(
+      Arrays.asList(WORDS_ARRAY), new Function<String, StringHolder>() {
+        @Override public StringHolder apply(String s) {
+          return new StringHolder(s);
+        }
+      });
+  private static final Set<StringHolder> EXPECTED_COUNT_SET =
+      ImmutableSet.copyOf(Lists.transform(
+          Arrays.asList("hi: 5", "there: 1", "sue: 2", "bob: 2"),
+          new Function<String, StringHolder>() {
+            @Override
+            public StringHolder apply(String s) {
+              return new StringHolder(s);
+            }
+          }));
+
+  @Test
+  public void testRun() throws Exception {
+    SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+    options.setRunner(SparkPipelineRunner.class);
+    Pipeline p = Pipeline.create(options);
+    PCollection<StringHolder> inputWords =
+        p.apply(Create.of(WORDS).withCoder(StringHolderUtf8Coder.of()));
+    PCollection<StringHolder> output = inputWords.apply(new CountWords());
+
+    DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET);
+
+    EvaluationResult res = SparkPipelineRunner.create().run(p);
+    res.close();
+  }
+
+  /**
+   * A DoFn that tokenizes lines of text into individual words.
+   */
+  static class ExtractWordsFn extends DoFn<StringHolder, StringHolder> {
+    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
+    private final Aggregator<Long, Long> emptyLines =
+        createAggregator("emptyLines", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      // Split the line into words.
+      String[] words = WORD_BOUNDARY.split(c.element().toString());
+
+      // Keep track of the number of lines without any words encountered while tokenizing.
+      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
+      if (words.length == 0) {
+        emptyLines.addValue(1L);
+      }
+
+      // Output each word encountered into the output PCollection.
+      for (String word : words) {
+        if (!word.isEmpty()) {
+          c.output(new StringHolder(word));
+        }
+      }
+    }
+  }
+
+  /**
+   * A DoFn that converts a Word and Count into a printable string.
+   */
+  private static class FormatCountsFn extends DoFn<KV<StringHolder, Long>, StringHolder> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(new StringHolder(c.element().getKey() + ": " + c.element().getValue()));
+    }
+  }
+
+  private static class CountWords extends PTransform<PCollection<StringHolder>, PCollection<StringHolder>> {
+    @Override
+    public PCollection<StringHolder> apply(PCollection<StringHolder> lines) {
+
+      // Convert lines of text into individual words.
+      PCollection<StringHolder> words = lines.apply(
+          ParDo.of(new ExtractWordsFn()));
+
+      // Count the number of times each word occurs.
+      PCollection<KV<StringHolder, Long>> wordCounts =
+          words.apply(Count.<StringHolder>perElement());
+
+      // Format each word and count into a printable string.
+
+      return wordCounts.apply(ParDo.of(new FormatCountsFn()));
+    }
+
+  }
+}



[12/23] incubator-beam git commit: [BEAM-11] Add Spark runner to runners module

Posted by am...@apache.org.
[BEAM-11] Add Spark runner to runners module


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

Branch: refs/heads/master
Commit: bde9933dcb0580d6ecc49968f4ba30f3151979ed
Parents: dd1c6cc
Author: Sela <an...@paypal.com>
Authored: Sat Mar 12 17:54:02 2016 +0200
Committer: Sela <an...@paypal.com>
Committed: Tue Mar 15 20:39:47 2016 +0200

----------------------------------------------------------------------
 runners/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bde9933d/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 4f07ceb..b2e9eb1 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -38,6 +38,7 @@
 
   <modules>
     <module>flink</module>
+    <module>spark</module>
   </modules>
 
 </project>


[09/23] incubator-beam git commit: [BEAM-11] Spark runner directory structure and pom setup.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
deleted file mode 100644
index 58b1924..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
+++ /dev/null
@@ -1,800 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.ImmutableMap;
-
-import org.apache.avro.mapred.AvroKey;
-import org.apache.avro.mapreduce.AvroJob;
-import org.apache.avro.mapreduce.AvroKeyInputFormat;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.Function2;
-import org.apache.spark.api.java.function.PairFunction;
-import scala.Tuple2;
-
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputDirectory;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputFilePrefix;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputFileTemplate;
-import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardCount;
-
-import com.cloudera.dataflow.hadoop.HadoopIO;
-
-/**
- * Supports translation between a DataFlow transform, and Spark's operations on RDDs.
- */
-public final class TransformTranslator {
-
-  private TransformTranslator() {
-  }
-
-  public static class FieldGetter {
-    private final Map<String, Field> fields;
-
-    public FieldGetter(Class<?> clazz) {
-      this.fields = Maps.newHashMap();
-      for (Field f : clazz.getDeclaredFields()) {
-        f.setAccessible(true);
-        this.fields.put(f.getName(), f);
-      }
-    }
-
-    public <T> T get(String fieldname, Object value) {
-      try {
-        @SuppressWarnings("unchecked")
-        T fieldValue = (T) fields.get(fieldname).get(value);
-        return fieldValue;
-      } catch (IllegalAccessException e) {
-        throw new IllegalStateException(e);
-      }
-    }
-  }
-
-  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
-    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
-        PCollectionList<T> pcs = context.getInput(transform);
-        JavaRDD<WindowedValue<T>>[] rdds = new JavaRDD[pcs.size()];
-        for (int i = 0; i < rdds.length; i++) {
-          rdds[i] = (JavaRDD<WindowedValue<T>>) context.getRDD(pcs.get(i));
-        }
-        JavaRDD<WindowedValue<T>> rdd = context.getSparkContext().union(rdds);
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>> gbk() {
-    return new TransformEvaluator<GroupByKey.GroupByKeyOnly<K, V>>() {
-      @Override
-      public void evaluate(GroupByKey.GroupByKeyOnly<K, V> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, V>>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context.getInputRDD(transform);
-        @SuppressWarnings("unchecked")
-        KvCoder<K, V> coder = (KvCoder<K, V>) context.getInput(transform).getCoder();
-        Coder<K> keyCoder = coder.getKeyCoder();
-        Coder<V> valueCoder = coder.getValueCoder();
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaRDDLike<WindowedValue<KV<K, Iterable<V>>>, ?> outRDD = fromPair(
-              toPair(inRDD.map(WindowingHelpers.<KV<K, V>>unwindowFunction()))
-            .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder))
-            .groupByKey()
-            .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder)))
-            // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
-            .map(WindowingHelpers.<KV<K, Iterable<V>>>windowFunction());
-        context.setOutputRDD(transform, outRDD);
-      }
-    };
-  }
-
-  private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class);
-
-  private static <K, VI, VO> TransformEvaluator<Combine.GroupedValues<K, VI, VO>> grouped() {
-    return new TransformEvaluator<Combine.GroupedValues<K, VI, VO>>() {
-      @Override
-      public void evaluate(Combine.GroupedValues<K, VI, VO> transform, EvaluationContext context) {
-        Combine.KeyedCombineFn<K, VI, ?, VO> keyed = GROUPED_FG.get("fn", transform);
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<KV<K, Iterable<VI>>>, ?>) context.getInputRDD(transform);
-        context.setOutputRDD(transform,
-            inRDD.map(new KVFunction<>(keyed)));
-      }
-    };
-  }
-
-  private static final FieldGetter COMBINE_GLOBALLY_FG = new FieldGetter(Combine.Globally.class);
-
-  private static <I, A, O> TransformEvaluator<Combine.Globally<I, O>> combineGlobally() {
-    return new TransformEvaluator<Combine.Globally<I, O>>() {
-
-      @Override
-      public void evaluate(Combine.Globally<I, O> transform, EvaluationContext context) {
-        final Combine.CombineFn<I, A, O> globally = COMBINE_GLOBALLY_FG.get("fn", transform);
-
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRdd =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-
-        final Coder<I> iCoder = context.getInput(transform).getCoder();
-        final Coder<A> aCoder;
-        try {
-          aCoder = globally.getAccumulatorCoder(
-              context.getPipeline().getCoderRegistry(), iCoder);
-        } catch (CannotProvideCoderException e) {
-          throw new IllegalStateException("Could not determine coder for accumulator", e);
-        }
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaRDD<byte[]> inRddBytes = inRdd
-            .map(WindowingHelpers.<I>unwindowFunction())
-            .map(CoderHelpers.toByteFunction(iCoder));
-
-        /*A*/ byte[] acc = inRddBytes.aggregate(
-            CoderHelpers.toByteArray(globally.createAccumulator(), aCoder),
-            new Function2</*A*/ byte[], /*I*/ byte[], /*A*/ byte[]>() {
-              @Override
-              public /*A*/ byte[] call(/*A*/ byte[] ab, /*I*/ byte[] ib) throws Exception {
-                A a = CoderHelpers.fromByteArray(ab, aCoder);
-                I i = CoderHelpers.fromByteArray(ib, iCoder);
-                return CoderHelpers.toByteArray(globally.addInput(a, i), aCoder);
-              }
-            },
-            new Function2</*A*/ byte[], /*A*/ byte[], /*A*/ byte[]>() {
-              @Override
-              public /*A*/ byte[] call(/*A*/ byte[] a1b, /*A*/ byte[] a2b) throws Exception {
-                A a1 = CoderHelpers.fromByteArray(a1b, aCoder);
-                A a2 = CoderHelpers.fromByteArray(a2b, aCoder);
-                // don't use Guava's ImmutableList.of as values may be null
-                List<A> accumulators = Collections.unmodifiableList(Arrays.asList(a1, a2));
-                A merged = globally.mergeAccumulators(accumulators);
-                return CoderHelpers.toByteArray(merged, aCoder);
-              }
-            }
-        );
-        O output = globally.extractOutput(CoderHelpers.fromByteArray(acc, aCoder));
-
-        Coder<O> coder = context.getOutput(transform).getCoder();
-        JavaRDD<byte[]> outRdd = context.getSparkContext().parallelize(
-            // don't use Guava's ImmutableList.of as output may be null
-            CoderHelpers.toByteArrays(Collections.singleton(output), coder));
-        context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder))
-            .map(WindowingHelpers.<O>windowFunction()));
-      }
-    };
-  }
-
-  private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class);
-
-  private static <K, VI, VA, VO> TransformEvaluator<Combine.PerKey<K, VI, VO>> combinePerKey() {
-    return new TransformEvaluator<Combine.PerKey<K, VI, VO>>() {
-      @Override
-      public void evaluate(Combine.PerKey<K, VI, VO> transform, EvaluationContext context) {
-        final Combine.KeyedCombineFn<K, VI, VA, VO> keyed =
-            COMBINE_PERKEY_FG.get("fn", transform);
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<KV<K, VI>>, ?> inRdd =
-            (JavaRDDLike<WindowedValue<KV<K, VI>>, ?>) context.getInputRDD(transform);
-
-        @SuppressWarnings("unchecked")
-        KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
-        Coder<K> keyCoder = inputCoder.getKeyCoder();
-        Coder<VI> viCoder = inputCoder.getValueCoder();
-        Coder<VA> vaCoder;
-        try {
-          vaCoder = keyed.getAccumulatorCoder(
-              context.getPipeline().getCoderRegistry(), keyCoder, viCoder);
-        } catch (CannotProvideCoderException e) {
-          throw new IllegalStateException("Could not determine coder for accumulator", e);
-        }
-        Coder<KV<K, VI>> kviCoder = KvCoder.of(keyCoder, viCoder);
-        Coder<KV<K, VA>> kvaCoder = KvCoder.of(keyCoder, vaCoder);
-
-        // We need to duplicate K as both the key of the JavaPairRDD as well as inside the value,
-        // since the functions passed to combineByKey don't receive the associated key of each
-        // value, and we need to map back into methods in Combine.KeyedCombineFn, which each
-        // require the key in addition to the VI's and VA's being merged/accumulated. Once Spark
-        // provides a way to include keys in the arguments of combine/merge functions, we won't
-        // need to duplicate the keys anymore.
-
-        // Key has to bw windowed in order to group by window as well
-        JavaPairRDD<WindowedValue<K>, WindowedValue<KV<K, VI>>> inRddDuplicatedKeyPair =
-            inRdd.mapToPair(
-                new PairFunction<WindowedValue<KV<K, VI>>, WindowedValue<K>,
-                    WindowedValue<KV<K, VI>>>() {
-                  @Override
-                  public Tuple2<WindowedValue<K>,
-                      WindowedValue<KV<K, VI>>> call(WindowedValue<KV<K, VI>> kv) {
-                    WindowedValue<K> wk = WindowedValue.of(kv.getValue().getKey(),
-                        kv.getTimestamp(), kv.getWindows(), kv.getPane());
-                    return new Tuple2<>(wk, kv);
-                  }
-                });
-        //-- windowed coders
-        final WindowedValue.FullWindowedValueCoder<K> wkCoder =
-                WindowedValue.FullWindowedValueCoder.of(keyCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-        final WindowedValue.FullWindowedValueCoder<KV<K, VI>> wkviCoder =
-                WindowedValue.FullWindowedValueCoder.of(kviCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-        final WindowedValue.FullWindowedValueCoder<KV<K, VA>> wkvaCoder =
-                WindowedValue.FullWindowedValueCoder.of(kvaCoder,
-                context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder());
-
-        // Use coders to convert objects in the PCollection to byte arrays, so they
-        // can be transferred over the network for the shuffle.
-        JavaPairRDD<ByteArray, byte[]> inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair
-            .mapToPair(CoderHelpers.toByteFunction(wkCoder, wkviCoder));
-
-        // The output of combineByKey will be "VA" (accumulator) types rather than "VO" (final
-        // output types) since Combine.CombineFn only provides ways to merge VAs, and no way
-        // to merge VOs.
-        JavaPairRDD</*K*/ ByteArray, /*KV<K, VA>*/ byte[]> accumulatedBytes =
-            inRddDuplicatedKeyPairBytes.combineByKey(
-            new Function</*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VI>*/ byte[] input) {
-                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
-                VA va = keyed.createAccumulator(wkvi.getValue().getKey());
-                va = keyed.addInput(wkvi.getValue().getKey(), va, wkvi.getValue().getValue());
-                WindowedValue<KV<K, VA>> wkva =
-                    WindowedValue.of(KV.of(wkvi.getValue().getKey(), va), wkvi.getTimestamp(),
-                    wkvi.getWindows(), wkvi.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            },
-            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VI>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc,
-                  /*KV<K, VI>*/ byte[] input) {
-                WindowedValue<KV<K, VA>> wkva = CoderHelpers.fromByteArray(acc, wkvaCoder);
-                WindowedValue<KV<K, VI>> wkvi = CoderHelpers.fromByteArray(input, wkviCoder);
-                VA va = keyed.addInput(wkva.getValue().getKey(), wkva.getValue().getValue(),
-                    wkvi.getValue().getValue());
-                wkva = WindowedValue.of(KV.of(wkva.getValue().getKey(), va), wkva.getTimestamp(),
-                    wkva.getWindows(), wkva.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            },
-            new Function2</*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[], /*KV<K, VA>*/ byte[]>() {
-              @Override
-              public /*KV<K, VA>*/ byte[] call(/*KV<K, VA>*/ byte[] acc1,
-                  /*KV<K, VA>*/ byte[] acc2) {
-                WindowedValue<KV<K, VA>> wkva1 = CoderHelpers.fromByteArray(acc1, wkvaCoder);
-                WindowedValue<KV<K, VA>> wkva2 = CoderHelpers.fromByteArray(acc2, wkvaCoder);
-                VA va = keyed.mergeAccumulators(wkva1.getValue().getKey(),
-                    // don't use Guava's ImmutableList.of as values may be null
-                    Collections.unmodifiableList(Arrays.asList(wkva1.getValue().getValue(),
-                    wkva2.getValue().getValue())));
-                WindowedValue<KV<K, VA>> wkva = WindowedValue.of(KV.of(wkva1.getValue().getKey(),
-                    va), wkva1.getTimestamp(), wkva1.getWindows(), wkva1.getPane());
-                return CoderHelpers.toByteArray(wkva, wkvaCoder);
-              }
-            });
-
-        JavaPairRDD<WindowedValue<K>, WindowedValue<VO>> extracted = accumulatedBytes
-            .mapToPair(CoderHelpers.fromByteFunction(wkCoder, wkvaCoder))
-            .mapValues(
-                new Function<WindowedValue<KV<K, VA>>, WindowedValue<VO>>() {
-                  @Override
-                  public WindowedValue<VO> call(WindowedValue<KV<K, VA>> acc) {
-                    return WindowedValue.of(keyed.extractOutput(acc.getValue().getKey(),
-                        acc.getValue().getValue()), acc.getTimestamp(),
-                        acc.getWindows(), acc.getPane());
-                  }
-                });
-
-        context.setOutputRDD(transform,
-            fromPair(extracted)
-            .map(new Function<KV<WindowedValue<K>, WindowedValue<VO>>, WindowedValue<KV<K, VO>>>() {
-              @Override
-              public WindowedValue<KV<K, VO>> call(KV<WindowedValue<K>, WindowedValue<VO>> kwvo)
-                  throws Exception {
-                WindowedValue<VO> wvo = kwvo.getValue();
-                KV<K, VO> kvo = KV.of(kwvo.getKey().getValue(), wvo.getValue());
-                return WindowedValue.of(kvo, wvo.getTimestamp(), wvo.getWindows(), wvo.getPane());
-              }
-            }));
-      }
-    };
-  }
-
-  private static final class KVFunction<K, VI, VO>
-      implements Function<WindowedValue<KV<K, Iterable<VI>>>, WindowedValue<KV<K, VO>>> {
-    private final Combine.KeyedCombineFn<K, VI, ?, VO> keyed;
-
-     KVFunction(Combine.KeyedCombineFn<K, VI, ?, VO> keyed) {
-      this.keyed = keyed;
-    }
-
-    @Override
-    public WindowedValue<KV<K, VO>> call(WindowedValue<KV<K, Iterable<VI>>> windowedKv)
-        throws Exception {
-      KV<K, Iterable<VI>> kv = windowedKv.getValue();
-      return WindowedValue.of(KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())),
-          windowedKv.getTimestamp(), windowedKv.getWindows(), windowedKv.getPane());
-    }
-  }
-
-  private static <K, V> JavaPairRDD<K, V> toPair(JavaRDDLike<KV<K, V>, ?> rdd) {
-    return rdd.mapToPair(new PairFunction<KV<K, V>, K, V>() {
-      @Override
-      public Tuple2<K, V> call(KV<K, V> kv) {
-        return new Tuple2<>(kv.getKey(), kv.getValue());
-      }
-    });
-  }
-
-  private static <K, V> JavaRDDLike<KV<K, V>, ?> fromPair(JavaPairRDD<K, V> rdd) {
-    return rdd.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-      @Override
-      public KV<K, V> call(Tuple2<K, V> t2) {
-        return KV.of(t2._1(), t2._2());
-      }
-    });
-  }
-
-  private static <I, O> TransformEvaluator<ParDo.Bound<I, O>> parDo() {
-    return new TransformEvaluator<ParDo.Bound<I, O>>() {
-      @Override
-      public void evaluate(ParDo.Bound<I, O> transform, EvaluationContext context) {
-        DoFnFunction<I, O> dofn =
-            new DoFnFunction<>(transform.getFn(),
-                context.getRuntimeContext(),
-                getSideInputs(transform.getSideInputs(), context));
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-        context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
-      }
-    };
-  }
-
-  private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class);
-
-  private static <I, O> TransformEvaluator<ParDo.BoundMulti<I, O>> multiDo() {
-    return new TransformEvaluator<ParDo.BoundMulti<I, O>>() {
-      @Override
-      public void evaluate(ParDo.BoundMulti<I, O> transform, EvaluationContext context) {
-        TupleTag<O> mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform);
-        MultiDoFnFunction<I, O> multifn = new MultiDoFnFunction<>(
-            transform.getFn(),
-            context.getRuntimeContext(),
-            mainOutputTag,
-            getSideInputs(transform.getSideInputs(), context));
-
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<I>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<I>, ?>) context.getInputRDD(transform);
-        JavaPairRDD<TupleTag<?>, WindowedValue<?>> all = inRDD
-            .mapPartitionsToPair(multifn)
-            .cache();
-
-        PCollectionTuple pct = context.getOutput(transform);
-        for (Map.Entry<TupleTag<?>, PCollection<?>> e : pct.getAll().entrySet()) {
-          @SuppressWarnings("unchecked")
-          JavaPairRDD<TupleTag<?>, WindowedValue<?>> filtered =
-              all.filter(new TupleTagFilter(e.getKey()));
-          @SuppressWarnings("unchecked")
-          // Object is the best we can do since different outputs can have different tags
-          JavaRDD<WindowedValue<Object>> values =
-              (JavaRDD<WindowedValue<Object>>) (JavaRDD<?>) filtered.values();
-          context.setRDD(e.getValue(), values);
-        }
-      }
-    };
-  }
-
-
-  private static <T> TransformEvaluator<TextIO.Read.Bound<T>> readText() {
-    return new TransformEvaluator<TextIO.Read.Bound<T>>() {
-      @Override
-      public void evaluate(TextIO.Read.Bound<T> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaRDD<WindowedValue<String>> rdd = context.getSparkContext().textFile(pattern)
-                .map(WindowingHelpers.<String>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<TextIO.Write.Bound<T>> writeText() {
-    return new TransformEvaluator<TextIO.Write.Bound<T>>() {
-      @Override
-      public void evaluate(TextIO.Write.Bound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<T, Void> last =
-            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
-            .map(WindowingHelpers.<T>unwindowFunction())
-            .mapToPair(new PairFunction<T, T,
-                    Void>() {
-              @Override
-              public Tuple2<T, Void> call(T t) throws Exception {
-                return new Tuple2<>(t, null);
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-                transform.getShardTemplate(), transform.getFilenamePrefix(),
-                transform.getFilenameSuffix());
-        writeHadoopFile(last, new Configuration(), shardTemplateInfo, Text.class,
-            NullWritable.class, TemplatedTextOutputFormat.class);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<AvroIO.Read.Bound<T>> readAvro() {
-    return new TransformEvaluator<AvroIO.Read.Bound<T>>() {
-      @Override
-      public void evaluate(AvroIO.Read.Bound<T> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaSparkContext jsc = context.getSparkContext();
-        @SuppressWarnings("unchecked")
-        JavaRDD<AvroKey<T>> avroFile = (JavaRDD<AvroKey<T>>) (JavaRDD<?>)
-            jsc.newAPIHadoopFile(pattern,
-                                 AvroKeyInputFormat.class,
-                                 AvroKey.class, NullWritable.class,
-                                 new Configuration()).keys();
-        JavaRDD<WindowedValue<T>> rdd = avroFile.map(
-            new Function<AvroKey<T>, T>() {
-              @Override
-              public T call(AvroKey<T> key) {
-                return key.datum();
-              }
-            }).map(WindowingHelpers.<T>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<AvroIO.Write.Bound<T>> writeAvro() {
-    return new TransformEvaluator<AvroIO.Write.Bound<T>>() {
-      @Override
-      public void evaluate(AvroIO.Write.Bound<T> transform, EvaluationContext context) {
-        Job job;
-        try {
-          job = Job.getInstance();
-        } catch (IOException e) {
-          throw new IllegalStateException(e);
-        }
-        AvroJob.setOutputKeySchema(job, transform.getSchema());
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<AvroKey<T>, NullWritable> last =
-            ((JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform))
-            .map(WindowingHelpers.<T>unwindowFunction())
-            .mapToPair(new PairFunction<T, AvroKey<T>, NullWritable>() {
-              @Override
-              public Tuple2<AvroKey<T>, NullWritable> call(T t) throws Exception {
-                return new Tuple2<>(new AvroKey<>(t), NullWritable.get());
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-            transform.getShardTemplate(), transform.getFilenamePrefix(),
-            transform.getFilenameSuffix());
-        writeHadoopFile(last, job.getConfiguration(), shardTemplateInfo,
-            AvroKey.class, NullWritable.class, TemplatedAvroKeyOutputFormat.class);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<HadoopIO.Read.Bound<K, V>> readHadoop() {
-    return new TransformEvaluator<HadoopIO.Read.Bound<K, V>>() {
-      @Override
-      public void evaluate(HadoopIO.Read.Bound<K, V> transform, EvaluationContext context) {
-        String pattern = transform.getFilepattern();
-        JavaSparkContext jsc = context.getSparkContext();
-        @SuppressWarnings ("unchecked")
-        JavaPairRDD<K, V> file = jsc.newAPIHadoopFile(pattern,
-            transform.getFormatClass(),
-            transform.getKeyClass(), transform.getValueClass(),
-            new Configuration());
-        JavaRDD<WindowedValue<KV<K, V>>> rdd =
-            file.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-          @Override
-          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
-            return KV.of(t2._1(), t2._2());
-          }
-        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
-        context.setOutputRDD(transform, rdd);
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<HadoopIO.Write.Bound<K, V>> writeHadoop() {
-    return new TransformEvaluator<HadoopIO.Write.Bound<K, V>>() {
-      @Override
-      public void evaluate(HadoopIO.Write.Bound<K, V> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaPairRDD<K, V> last = ((JavaRDDLike<WindowedValue<KV<K, V>>, ?>) context
-            .getInputRDD(transform))
-            .map(WindowingHelpers.<KV<K, V>>unwindowFunction())
-            .mapToPair(new PairFunction<KV<K, V>, K, V>() {
-              @Override
-              public Tuple2<K, V> call(KV<K, V> t) throws Exception {
-                return new Tuple2<>(t.getKey(), t.getValue());
-              }
-            });
-        ShardTemplateInformation shardTemplateInfo =
-            new ShardTemplateInformation(transform.getNumShards(),
-                transform.getShardTemplate(), transform.getFilenamePrefix(),
-                transform.getFilenameSuffix());
-        Configuration conf = new Configuration();
-        for (Map.Entry<String, String> e : transform.getConfigurationProperties().entrySet()) {
-          conf.set(e.getKey(), e.getValue());
-        }
-        writeHadoopFile(last, conf, shardTemplateInfo,
-            transform.getKeyClass(), transform.getValueClass(), transform.getFormatClass());
-      }
-    };
-  }
-
-  private static final class ShardTemplateInformation {
-    private final int numShards;
-    private final String shardTemplate;
-    private final String filenamePrefix;
-    private final String filenameSuffix;
-
-    private ShardTemplateInformation(int numShards, String shardTemplate, String
-        filenamePrefix, String filenameSuffix) {
-      this.numShards = numShards;
-      this.shardTemplate = shardTemplate;
-      this.filenamePrefix = filenamePrefix;
-      this.filenameSuffix = filenameSuffix;
-    }
-
-    int getNumShards() {
-      return numShards;
-    }
-
-    String getShardTemplate() {
-      return shardTemplate;
-    }
-
-    String getFilenamePrefix() {
-      return filenamePrefix;
-    }
-
-    String getFilenameSuffix() {
-      return filenameSuffix;
-    }
-  }
-
-  private static <K, V> void writeHadoopFile(JavaPairRDD<K, V> rdd, Configuration conf,
-      ShardTemplateInformation shardTemplateInfo, Class<?> keyClass, Class<?> valueClass,
-      Class<? extends FileOutputFormat> formatClass) {
-    int numShards = shardTemplateInfo.getNumShards();
-    String shardTemplate = shardTemplateInfo.getShardTemplate();
-    String filenamePrefix = shardTemplateInfo.getFilenamePrefix();
-    String filenameSuffix = shardTemplateInfo.getFilenameSuffix();
-    if (numShards != 0) {
-      // number of shards was set explicitly, so repartition
-      rdd = rdd.repartition(numShards);
-    }
-    int actualNumShards = rdd.partitions().size();
-    String template = replaceShardCount(shardTemplate, actualNumShards);
-    String outputDir = getOutputDirectory(filenamePrefix, template);
-    String filePrefix = getOutputFilePrefix(filenamePrefix, template);
-    String fileTemplate = getOutputFileTemplate(filenamePrefix, template);
-
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_PREFIX, filePrefix);
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_TEMPLATE, fileTemplate);
-    conf.set(ShardNameTemplateHelper.OUTPUT_FILE_SUFFIX, filenameSuffix);
-    rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
-  }
-
-  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
-
-  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
-    return new TransformEvaluator<Window.Bound<T>>() {
-      @Override
-      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaRDDLike<WindowedValue<T>, ?> inRDD =
-            (JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform);
-        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
-        if (windowFn instanceof GlobalWindows) {
-          context.setOutputRDD(transform, inRDD);
-        } else {
-          @SuppressWarnings("unchecked")
-          DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
-          DoFnFunction<T, T> dofn =
-                  new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null);
-          context.setOutputRDD(transform, inRDD.mapPartitions(dofn));
-        }
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Create.Values<T>> create() {
-    return new TransformEvaluator<Create.Values<T>>() {
-      @Override
-      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
-        Iterable<T> elems = transform.getElements();
-        // Use a coder to convert the objects in the PCollection to byte arrays, so they
-        // can be transferred over the network.
-        Coder<T> coder = context.getOutput(transform).getCoder();
-        context.setOutputRDDFromValues(transform, elems, coder);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<View.AsSingleton<T>> viewAsSingleton() {
-    return new TransformEvaluator<View.AsSingleton<T>>() {
-      @Override
-      public void evaluate(View.AsSingleton<T> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-                context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<View.AsIterable<T>> viewAsIter() {
-    return new TransformEvaluator<View.AsIterable<T>>() {
-      @Override
-      public void evaluate(View.AsIterable<T> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-                context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static <R, W> TransformEvaluator<View.CreatePCollectionView<R, W>> createPCollView() {
-    return new TransformEvaluator<View.CreatePCollectionView<R, W>>() {
-      @Override
-      public void evaluate(View.CreatePCollectionView<R, W> transform, EvaluationContext context) {
-        Iterable<? extends WindowedValue<?>> iter =
-            context.getWindowedValues(context.getInput(transform));
-        context.setPView(context.getOutput(transform), iter);
-      }
-    };
-  }
-
-  private static final class TupleTagFilter<V>
-      implements Function<Tuple2<TupleTag<V>, WindowedValue<?>>, Boolean> {
-
-    private final TupleTag<V> tag;
-
-    private TupleTagFilter(TupleTag<V> tag) {
-      this.tag = tag;
-    }
-
-    @Override
-    public Boolean call(Tuple2<TupleTag<V>, WindowedValue<?>> input) {
-      return tag.equals(input._1());
-    }
-  }
-
-  private static Map<TupleTag<?>, BroadcastHelper<?>> getSideInputs(
-      List<PCollectionView<?>> views,
-      EvaluationContext context) {
-    if (views == null) {
-      return ImmutableMap.of();
-    } else {
-      Map<TupleTag<?>, BroadcastHelper<?>> sideInputs = Maps.newHashMap();
-      for (PCollectionView<?> view : views) {
-        Iterable<? extends WindowedValue<?>> collectionView = context.getPCollectionView(view);
-        Coder<Iterable<WindowedValue<?>>> coderInternal = view.getCoderInternal();
-        @SuppressWarnings("unchecked")
-        BroadcastHelper<?> helper =
-            BroadcastHelper.create((Iterable<WindowedValue<?>>) collectionView, coderInternal);
-        //broadcast side inputs
-        helper.broadcast(context.getSparkContext());
-        sideInputs.put(view.getTagInternal(), helper);
-      }
-      return sideInputs;
-    }
-  }
-
-  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
-      .newHashMap();
-
-  static {
-    EVALUATORS.put(TextIO.Read.Bound.class, readText());
-    EVALUATORS.put(TextIO.Write.Bound.class, writeText());
-    EVALUATORS.put(AvroIO.Read.Bound.class, readAvro());
-    EVALUATORS.put(AvroIO.Write.Bound.class, writeAvro());
-    EVALUATORS.put(HadoopIO.Read.Bound.class, readHadoop());
-    EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop());
-    EVALUATORS.put(ParDo.Bound.class, parDo());
-    EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
-    EVALUATORS.put(GroupByKey.GroupByKeyOnly.class, gbk());
-    EVALUATORS.put(Combine.GroupedValues.class, grouped());
-    EVALUATORS.put(Combine.Globally.class, combineGlobally());
-    EVALUATORS.put(Combine.PerKey.class, combinePerKey());
-    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
-    EVALUATORS.put(Create.Values.class, create());
-    EVALUATORS.put(View.AsSingleton.class, viewAsSingleton());
-    EVALUATORS.put(View.AsIterable.class, viewAsIter());
-    EVALUATORS.put(View.CreatePCollectionView.class, createPCollView());
-    EVALUATORS.put(Window.Bound.class, window());
-  }
-
-  public static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
-  getTransformEvaluator(Class<PT> clazz) {
-    @SuppressWarnings("unchecked")
-    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
-    if (transform == null) {
-      throw new IllegalStateException("No TransformEvaluator registered for " + clazz);
-    }
-    return transform;
-  }
-
-  /**
-   * Translator matches Dataflow transformation with the appropriate evaluator.
-   */
-  public static class Translator implements SparkPipelineTranslator {
-
-    @Override
-    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
-      return EVALUATORS.containsKey(clazz);
-    }
-
-    @Override
-    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
-      return getTransformEvaluator(clazz);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/WindowingHelpers.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/WindowingHelpers.java
deleted file mode 100644
index 90600b2..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/WindowingHelpers.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark;
-
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import org.apache.spark.api.java.function.Function;
-
-/**
- * Helper functions for working with windows.
- */
-public final class WindowingHelpers {
-  private WindowingHelpers() {
-  }
-
-  /**
-   * A function for converting a value to a {@link WindowedValue}. The resulting
-   * {@link WindowedValue} will be in no windows, and will have the default timestamp
-   * and pane.
-   *
-   * @param <T>   The type of the object.
-   * @return A function that accepts an object and returns its {@link WindowedValue}.
-   */
-  public static <T> Function<T, WindowedValue<T>> windowFunction() {
-    return new Function<T, WindowedValue<T>>() {
-      @Override
-      public WindowedValue<T> call(T t) {
-        return WindowedValue.valueInEmptyWindows(t);
-      }
-    };
-  }
-
-  /**
-   * A function for extracting the value from a {@link WindowedValue}.
-   *
-   * @param <T>   The type of the object.
-   * @return A function that accepts a {@link WindowedValue} and returns its value.
-   */
-  public static <T> Function<WindowedValue<T>, T> unwindowFunction() {
-    return new Function<WindowedValue<T>, T>() {
-      @Override
-      public T call(WindowedValue<T> t) {
-        return t.getValue();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java
deleted file mode 100644
index a3055a2..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.aggregators;
-
-import org.apache.spark.AccumulatorParam;
-
-public class AggAccumParam implements AccumulatorParam<NamedAggregators> {
-  @Override
-  public NamedAggregators addAccumulator(NamedAggregators current, NamedAggregators added) {
-    return current.merge(added);
-  }
-
-  @Override
-  public NamedAggregators addInPlace(NamedAggregators current, NamedAggregators added) {
-    return addAccumulator(current, added);
-  }
-
-  @Override
-  public NamedAggregators zero(NamedAggregators initialValue) {
-    return new NamedAggregators();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java
deleted file mode 100644
index d51e404..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.aggregators;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.util.Map;
-import java.util.TreeMap;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.common.collect.ImmutableList;
-
-import com.cloudera.dataflow.spark.SparkRuntimeContext;
-
-/**
- * This class wraps a map of named aggregators. Spark expects that all accumulators be declared
- * before a job is launched. Dataflow allows aggregators to be used and incremented on the fly.
- * We create a map of named aggregators and instantiate in the the spark context before the job
- * is launched. We can then add aggregators on the fly in Spark.
- */
-public class NamedAggregators implements Serializable {
-  /**
-   * Map from aggregator name to current state.
-   */
-  private final Map<String, State<?, ?, ?>> mNamedAggregators = new TreeMap<>();
-
-  /**
-   * Constructs a new NamedAggregators instance.
-   */
-  public NamedAggregators() {
-  }
-
-  /**
-   * Constructs a new named aggregators instance that contains a mapping from the specified
-   * `named` to the associated initial state.
-   *
-   * @param name  Name of aggregator.
-   * @param state Associated State.
-   */
-  public NamedAggregators(String name, State<?, ?, ?> state) {
-    this.mNamedAggregators.put(name, state);
-  }
-
-  /**
-   * @param name      Name of aggregator to retrieve.
-   * @param typeClass Type class to cast the value to.
-   * @param <T>       Type to be returned.
-   * @return the value of the aggregator associated with the specified name
-   */
-  public <T> T getValue(String name, Class<T> typeClass) {
-    return typeClass.cast(mNamedAggregators.get(name).render());
-  }
-
-  /**
-   * Merges another NamedAggregators instance with this instance.
-   *
-   * @param other The other instance of named aggregators ot merge.
-   * @return This instance of Named aggregators with associated states updated to reflect the
-   * other instance's aggregators.
-   */
-  public NamedAggregators merge(NamedAggregators other) {
-    for (Map.Entry<String, State<?, ?, ?>> e : other.mNamedAggregators.entrySet()) {
-      String key = e.getKey();
-      State<?, ?, ?> otherValue = e.getValue();
-      State<?, ?, ?> value = mNamedAggregators.get(key);
-      if (value == null) {
-        mNamedAggregators.put(key, otherValue);
-      } else {
-        mNamedAggregators.put(key, merge(value, otherValue));
-      }
-    }
-    return this;
-  }
-
-  /**
-   * Helper method to merge States whose generic types aren't provably the same,
-   * so require some casting.
-   */
-  @SuppressWarnings("unchecked")
-  private static <A, B, C> State<A, B, C> merge(State<?, ?, ?> s1, State<?, ?, ?> s2) {
-    return ((State<A, B, C>) s1).merge((State<A, B, C>) s2);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    for (Map.Entry<String, State<?, ?, ?>> e : mNamedAggregators.entrySet()) {
-      sb.append(e.getKey()).append(": ").append(e.getValue().render());
-    }
-    return sb.toString();
-  }
-
-  /**
-   * @param <IN>    Input data type
-   * @param <INTER> Intermediate data type (useful for averages)
-   * @param <OUT>   Output data type
-   */
-  public interface State<IN, INTER, OUT> extends Serializable {
-    /**
-     * @param element new element to update state
-     */
-    void update(IN element);
-
-    State<IN, INTER, OUT> merge(State<IN, INTER, OUT> other);
-
-    INTER current();
-
-    OUT render();
-
-    Combine.CombineFn<IN, INTER, OUT> getCombineFn();
-  }
-
-  /**
-   * =&gt; combineFunction in data flow.
-   */
-  public static class CombineFunctionState<IN, INTER, OUT> implements State<IN, INTER, OUT> {
-
-    private Combine.CombineFn<IN, INTER, OUT> combineFn;
-    private Coder<IN> inCoder;
-    private SparkRuntimeContext ctxt;
-    private transient INTER state;
-
-    public CombineFunctionState(
-        Combine.CombineFn<IN, INTER, OUT> combineFn,
-        Coder<IN> inCoder,
-        SparkRuntimeContext ctxt) {
-      this.combineFn = combineFn;
-      this.inCoder = inCoder;
-      this.ctxt = ctxt;
-      this.state = combineFn.createAccumulator();
-    }
-
-    @Override
-    public void update(IN element) {
-      combineFn.addInput(state, element);
-    }
-
-    @Override
-    public State<IN, INTER, OUT> merge(State<IN, INTER, OUT> other) {
-      this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current()));
-      return this;
-    }
-
-    @Override
-    public INTER current() {
-      return state;
-    }
-
-    @Override
-    public OUT render() {
-      return combineFn.extractOutput(state);
-    }
-
-    @Override
-    public Combine.CombineFn<IN, INTER, OUT> getCombineFn() {
-      return combineFn;
-    }
-
-    private void writeObject(ObjectOutputStream oos) throws IOException {
-      oos.writeObject(ctxt);
-      oos.writeObject(combineFn);
-      oos.writeObject(inCoder);
-      try {
-        combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder)
-            .encode(state, oos, Coder.Context.NESTED);
-      } catch (CannotProvideCoderException e) {
-        throw new IllegalStateException("Could not determine coder for accumulator", e);
-      }
-    }
-
-    @SuppressWarnings("unchecked")
-    private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
-      ctxt = (SparkRuntimeContext) ois.readObject();
-      combineFn = (Combine.CombineFn<IN, INTER, OUT>) ois.readObject();
-      inCoder = (Coder<IN>) ois.readObject();
-      try {
-        state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder)
-            .decode(ois, Coder.Context.NESTED);
-      } catch (CannotProvideCoderException e) {
-        throw new IllegalStateException("Could not determine coder for accumulator", e);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptions.java
deleted file mode 100644
index 57253f0..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptions.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-
-import com.cloudera.dataflow.spark.SparkPipelineOptions;
-
-/**
- * Options used to configure Spark streaming.
- */
-public interface SparkStreamingPipelineOptions extends SparkPipelineOptions {
-  @Description("Timeout to wait (in msec) for the streaming execution so stop, -1 runs until " +
-          "execution is stopped")
-  @Default.Long(-1)
-  Long getTimeout();
-
-  void setTimeout(Long batchInterval);
-
-  @Override
-  @Default.Boolean(true)
-  boolean isStreaming();
-
-  @Override
-  @Default.String("spark streaming dataflow pipeline job")
-  String getAppName();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsFactory.java
deleted file mode 100644
index 3b568af..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsFactory.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-
-public final class SparkStreamingPipelineOptionsFactory {
-
-  private SparkStreamingPipelineOptionsFactory() {
-  }
-
-  public static SparkStreamingPipelineOptions create() {
-    return PipelineOptionsFactory.as(SparkStreamingPipelineOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
deleted file mode 100644
index 01c4375..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-public class SparkStreamingPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
-
-  @Override
-  public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-    return ImmutableList.<Class<? extends PipelineOptions>>of(SparkStreamingPipelineOptions
-            .class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java
deleted file mode 100644
index 5ecd562..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-
-
-import com.cloudera.dataflow.spark.EvaluationContext;
-import com.cloudera.dataflow.spark.SparkRuntimeContext;
-
-/**
- * Streaming evaluation context helps to handle streaming.
- */
-public class StreamingEvaluationContext extends EvaluationContext {
-
-  private final JavaStreamingContext jssc;
-  private final long timeout;
-  private final Map<PValue, DStreamHolder<?>> pstreams = new LinkedHashMap<>();
-  private final Set<DStreamHolder<?>> leafStreams = new LinkedHashSet<>();
-
-  public StreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
-      JavaStreamingContext jssc, long timeout) {
-    super(jsc, pipeline);
-    this.jssc = jssc;
-    this.timeout = timeout;
-  }
-
-  /**
-   * DStream holder Can also crate a DStream from a supplied queue of values, but mainly for
-   * testing.
-   */
-  private class DStreamHolder<T> {
-
-    private Iterable<Iterable<T>> values;
-    private Coder<T> coder;
-    private JavaDStream<WindowedValue<T>> dStream;
-
-    DStreamHolder(Iterable<Iterable<T>> values, Coder<T> coder) {
-      this.values = values;
-      this.coder = coder;
-    }
-
-    DStreamHolder(JavaDStream<WindowedValue<T>> dStream) {
-      this.dStream = dStream;
-    }
-
-    @SuppressWarnings("unchecked")
-    JavaDStream<WindowedValue<T>> getDStream() {
-      if (dStream == null) {
-        // create the DStream from values
-        Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
-        for (Iterable<T> v : values) {
-          setOutputRDDFromValues(currentTransform.getTransform(), v, coder);
-          rddQueue.offer((JavaRDD<WindowedValue<T>>) getOutputRDD(currentTransform.getTransform()));
-        }
-        // 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);
-      }
-      return dStream;
-    }
-  }
-
-  <T> void setDStreamFromQueue(
-      PTransform<?, ?> transform, Iterable<Iterable<T>> values, Coder<T> coder) {
-    pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder));
-  }
-
-  <T> void setStream(PTransform<?, ?> transform, JavaDStream<WindowedValue<T>> dStream) {
-    PValue pvalue = (PValue) getOutput(transform);
-    DStreamHolder<T> dStreamHolder = new DStreamHolder<>(dStream);
-    pstreams.put(pvalue, dStreamHolder);
-    leafStreams.add(dStreamHolder);
-  }
-
-  boolean hasStream(PTransform<?, ?> transform) {
-    PValue pvalue = (PValue) getInput(transform);
-    return pstreams.containsKey(pvalue);
-  }
-
-  JavaDStreamLike<?, ?, ?> getStream(PTransform<?, ?> transform) {
-    return getStream((PValue) getInput(transform));
-  }
-
-  JavaDStreamLike<?, ?, ?> getStream(PValue pvalue) {
-    DStreamHolder<?> dStreamHolder = pstreams.get(pvalue);
-    JavaDStreamLike<?, ?, ?> dStream = dStreamHolder.getDStream();
-    leafStreams.remove(dStreamHolder);
-    return dStream;
-  }
-
-  // used to set the RDD from the DStream in the RDDHolder for transformation
-  <T> void setInputRDD(
-      PTransform<? extends PInput, ?> transform, JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    setRDD((PValue) getInput(transform), rdd);
-  }
-
-  // used to get the RDD transformation output and use it as the DStream transformation output
-  JavaRDDLike<?, ?> getOutputRDD(PTransform<?, ?> transform) {
-    return getRDD((PValue) getOutput(transform));
-  }
-
-  public JavaStreamingContext getStreamingContext() {
-    return jssc;
-  }
-
-  @Override
-  protected void computeOutputs() {
-    for (DStreamHolder<?> streamHolder : leafStreams) {
-      computeOutput(streamHolder);
-    }
-  }
-
-  private static <T> void computeOutput(DStreamHolder<T> streamHolder) {
-    streamHolder.getDStream().foreachRDD(new Function<JavaRDD<WindowedValue<T>>, Void>() {
-      @Override
-      public Void call(JavaRDD<WindowedValue<T>> rdd) throws Exception {
-        rdd.rdd().cache();
-        rdd.count();
-        return null;
-      }
-    }); // force a DStream action
-  }
-
-  @Override
-  public void close() {
-    if (timeout > 0) {
-      jssc.awaitTerminationOrTimeout(timeout);
-    } else {
-      jssc.awaitTermination();
-    }
-    //TODO: stop gracefully ?
-    jssc.stop(false, false);
-    state = State.DONE;
-    super.close();
-  }
-
-  private State state = State.RUNNING;
-
-  @Override
-  public State getState() {
-    return state;
-  }
-
-  //---------------- override in order to expose in package
-  @Override
-  protected <I extends PInput> I getInput(PTransform<I, ?> transform) {
-    return super.getInput(transform);
-  }
-  @Override
-  protected <O extends POutput> O getOutput(PTransform<?, O> transform) {
-    return super.getOutput(transform);
-  }
-
-  @Override
-  protected JavaSparkContext getSparkContext() {
-    return super.getSparkContext();
-  }
-
-  @Override
-  protected SparkRuntimeContext getRuntimeContext() {
-    return super.getRuntimeContext();
-  }
-
-  @Override
-  protected void setCurrentTransform(AppliedPTransform<?, ?, ?> transform) {
-    super.setCurrentTransform(transform);
-  }
-
-  @Override
-  protected AppliedPTransform<?, ?, ?> getCurrentTransform() {
-    return super.getCurrentTransform();
-  }
-
-  @Override
-  protected <T> void setOutputRDD(PTransform<?, ?> transform,
-      JavaRDDLike<WindowedValue<T>, ?> rdd) {
-    super.setOutputRDD(transform, rdd);
-  }
-
-  @Override
-  protected <T> void setOutputRDDFromValues(PTransform<?, ?> transform, Iterable<T> values,
-      Coder<T> coder) {
-    super.setOutputRDDFromValues(transform, values, coder);
-  }
-
-  @Override
-  protected boolean hasOutputRDD(PTransform<? extends PInput, ?> transform) {
-    return super.hasOutputRDD(transform);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java
deleted file mode 100644
index d8ae5e8..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java
+++ /dev/null
@@ -1,414 +0,0 @@
-/*
- * Copyright (c) 2015, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.api.client.util.Lists;
-import com.google.api.client.util.Maps;
-import com.google.api.client.util.Sets;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.repackaged.com.google.common.reflect.TypeToken;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-import kafka.serializer.Decoder;
-
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.Durations;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
-import org.apache.spark.streaming.api.java.JavaPairInputDStream;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.kafka.KafkaUtils;
-
-import scala.Tuple2;
-
-import com.cloudera.dataflow.hadoop.HadoopIO;
-import com.cloudera.dataflow.io.ConsoleIO;
-import com.cloudera.dataflow.io.CreateStream;
-import com.cloudera.dataflow.io.KafkaIO;
-import com.cloudera.dataflow.spark.DoFnFunction;
-import com.cloudera.dataflow.spark.EvaluationContext;
-import com.cloudera.dataflow.spark.SparkPipelineTranslator;
-import com.cloudera.dataflow.spark.TransformEvaluator;
-import com.cloudera.dataflow.spark.TransformTranslator;
-import com.cloudera.dataflow.spark.WindowingHelpers;
-
-/**
- * Supports translation between a DataFlow transform, and Spark's operations on DStreams.
- */
-public final class StreamingTransformTranslator {
-
-  private StreamingTransformTranslator() {
-  }
-
-  private static <T> TransformEvaluator<ConsoleIO.Write.Unbound<T>> print() {
-    return new TransformEvaluator<ConsoleIO.Write.Unbound<T>>() {
-      @Override
-      public void evaluate(ConsoleIO.Write.Unbound<T> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
-            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
-            ((StreamingEvaluationContext) context).getStream(transform);
-        dstream.map(WindowingHelpers.<T>unwindowFunction()).print(transform.getNum());
-      }
-    };
-  }
-
-  private static <K, V> TransformEvaluator<KafkaIO.Read.Unbound<K, V>> kafka() {
-    return new TransformEvaluator<KafkaIO.Read.Unbound<K, V>>() {
-      @Override
-      public void evaluate(KafkaIO.Read.Unbound<K, V> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        JavaStreamingContext jssc = sec.getStreamingContext();
-        Class<K> keyClazz = transform.getKeyClass();
-        Class<V> valueClazz = transform.getValueClass();
-        Class<? extends Decoder<K>> keyDecoderClazz = transform.getKeyDecoderClass();
-        Class<? extends Decoder<V>> valueDecoderClazz = transform.getValueDecoderClass();
-        Map<String, String> kafkaParams = transform.getKafkaParams();
-        Set<String> topics = transform.getTopics();
-        JavaPairInputDStream<K, V> inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz,
-                valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics);
-        JavaDStream<WindowedValue<KV<K, V>>> inputStream =
-            inputPairStream.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-          @Override
-          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
-            return KV.of(t2._1(), t2._2());
-          }
-        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
-        sec.setStream(transform, inputStream);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Create.Values<T>> create() {
-    return new TransformEvaluator<Create.Values<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Create.Values<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        Iterable<T> elems = transform.getElements();
-        Coder<T> coder = sec.getOutput(transform).getCoder();
-        if (coder != VoidCoder.of()) {
-          // actual create
-          sec.setOutputRDDFromValues(transform, elems, coder);
-        } else {
-          // fake create as an input
-          // creates a stream with a single batch containing a single null element
-          // to invoke following transformations once
-          // to support DataflowAssert
-          sec.setDStreamFromQueue(transform,
-              Collections.<Iterable<Void>>singletonList(Collections.singletonList((Void) null)),
-              (Coder<Void>) coder);
-        }
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<CreateStream.QueuedValues<T>> createFromQueue() {
-    return new TransformEvaluator<CreateStream.QueuedValues<T>>() {
-      @Override
-      public void evaluate(CreateStream.QueuedValues<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        Iterable<Iterable<T>> values = transform.getQueuedValues();
-        Coder<T> coder = sec.getOutput(transform).getCoder();
-        sec.setDStreamFromQueue(transform, values, coder);
-      }
-    };
-  }
-
-  private static <T> TransformEvaluator<Flatten.FlattenPCollectionList<T>> flattenPColl() {
-    return new TransformEvaluator<Flatten.FlattenPCollectionList<T>>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(Flatten.FlattenPCollectionList<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        PCollectionList<T> pcs = sec.getInput(transform);
-        JavaDStream<WindowedValue<T>> first =
-            (JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(0));
-        List<JavaDStream<WindowedValue<T>>> rest = Lists.newArrayListWithCapacity(pcs.size() - 1);
-        for (int i = 1; i < pcs.size(); i++) {
-          rest.add((JavaDStream<WindowedValue<T>>) sec.getStream(pcs.get(i)));
-        }
-        JavaDStream<WindowedValue<T>> dstream = sec.getStreamingContext().union(first, rest);
-        sec.setStream(transform, dstream);
-      }
-    };
-  }
-
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> rddTransform(
-      final SparkPipelineTranslator rddTranslator) {
-    return new TransformEvaluator<PT>() {
-      @SuppressWarnings("unchecked")
-      @Override
-      public void evaluate(PT transform, EvaluationContext context) {
-        TransformEvaluator<PT> rddEvaluator =
-            rddTranslator.translate((Class<PT>) transform.getClass());
-
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        if (sec.hasStream(transform)) {
-          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
-              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
-              sec.getStream(transform);
-
-          sec.setStream(transform, dStream
-              .transform(new RDDTransform<>(sec, rddEvaluator, transform)));
-        } else {
-          // if the transformation requires direct access to RDD (not in stream)
-          // this is used for "fake" transformations like with DataflowAssert
-          rddEvaluator.evaluate(transform, context);
-        }
-      }
-    };
-  }
-
-  /**
-   * RDD transform function If the transformation function doesn't have an input, create a fake one
-   * as an empty RDD.
-   *
-   * @param <PT> PTransform type
-   */
-  private static final class RDDTransform<PT extends PTransform<?, ?>>
-      implements Function<JavaRDD<WindowedValue<Object>>, JavaRDD<WindowedValue<Object>>> {
-
-    private final StreamingEvaluationContext context;
-    private final AppliedPTransform<?, ?, ?> appliedPTransform;
-    private final TransformEvaluator<PT> rddEvaluator;
-    private final PT transform;
-
-
-    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator<PT> rddEvaluator,
-        PT transform) {
-      this.context = context;
-      this.appliedPTransform = context.getCurrentTransform();
-      this.rddEvaluator = rddEvaluator;
-      this.transform = transform;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public JavaRDD<WindowedValue<Object>>
-        call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
-      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
-      context.setCurrentTransform(appliedPTransform);
-      context.setInputRDD(transform, rdd);
-      rddEvaluator.evaluate(transform, context);
-      if (!context.hasOutputRDD(transform)) {
-        // fake RDD as output
-        context.setOutputRDD(transform,
-            context.getSparkContext().<WindowedValue<Object>>emptyRDD());
-      }
-      JavaRDD<WindowedValue<Object>> outRDD =
-          (JavaRDD<WindowedValue<Object>>) context.getOutputRDD(transform);
-      context.setCurrentTransform(existingAPT);
-      return outRDD;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT> foreachRDD(
-      final SparkPipelineTranslator rddTranslator) {
-    return new TransformEvaluator<PT>() {
-      @Override
-      public void evaluate(PT transform, EvaluationContext context) {
-        TransformEvaluator<PT> rddEvaluator =
-            rddTranslator.translate((Class<PT>) transform.getClass());
-
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        if (sec.hasStream(transform)) {
-          JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>> dStream =
-              (JavaDStreamLike<WindowedValue<Object>, ?, JavaRDD<WindowedValue<Object>>>)
-              sec.getStream(transform);
-
-          dStream.foreachRDD(new RDDOutputOperator<>(sec, rddEvaluator, transform));
-        } else {
-          rddEvaluator.evaluate(transform, context);
-        }
-      }
-    };
-  }
-
-  /**
-   * RDD output function.
-   *
-   * @param <PT> PTransform type
-   */
-  private static final class RDDOutputOperator<PT extends PTransform<?, ?>>
-      implements Function<JavaRDD<WindowedValue<Object>>, Void> {
-
-    private final StreamingEvaluationContext context;
-    private final AppliedPTransform<?, ?, ?> appliedPTransform;
-    private final TransformEvaluator<PT> rddEvaluator;
-    private final PT transform;
-
-
-    private RDDOutputOperator(StreamingEvaluationContext context,
-        TransformEvaluator<PT> rddEvaluator, PT transform) {
-      this.context = context;
-      this.appliedPTransform = context.getCurrentTransform();
-      this.rddEvaluator = rddEvaluator;
-      this.transform = transform;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public Void call(JavaRDD<WindowedValue<Object>> rdd) throws Exception {
-      AppliedPTransform<?, ?, ?> existingAPT = context.getCurrentTransform();
-      context.setCurrentTransform(appliedPTransform);
-      context.setInputRDD(transform, rdd);
-      rddEvaluator.evaluate(transform, context);
-      context.setCurrentTransform(existingAPT);
-      return null;
-    }
-  }
-
-  private static final TransformTranslator.FieldGetter WINDOW_FG =
-      new TransformTranslator.FieldGetter(Window.Bound.class);
-
-  private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
-    return new TransformEvaluator<Window.Bound<T>>() {
-      @Override
-      public void evaluate(Window.Bound<T> transform, EvaluationContext context) {
-        StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        //--- first we apply windowing to the stream
-        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
-        @SuppressWarnings("unchecked")
-        JavaDStream<WindowedValue<T>> dStream =
-            (JavaDStream<WindowedValue<T>>) sec.getStream(transform);
-        if (windowFn instanceof FixedWindows) {
-          Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize()
-              .getMillis());
-          sec.setStream(transform, dStream.window(windowDuration));
-        } else if (windowFn instanceof SlidingWindows) {
-          Duration windowDuration = Durations.milliseconds(((SlidingWindows) windowFn).getSize()
-              .getMillis());
-          Duration slideDuration = Durations.milliseconds(((SlidingWindows) windowFn).getPeriod()
-              .getMillis());
-          sec.setStream(transform, dStream.window(windowDuration, slideDuration));
-        }
-        //--- then we apply windowing to the elements
-        DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
-        DoFnFunction<T, T> dofn = new DoFnFunction<>(addWindowsDoFn,
-            ((StreamingEvaluationContext)context).getRuntimeContext(), null);
-        @SuppressWarnings("unchecked")
-        JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>> dstream =
-            (JavaDStreamLike<WindowedValue<T>, ?, JavaRDD<WindowedValue<T>>>)
-            sec.getStream(transform);
-        sec.setStream(transform, dstream.mapPartitions(dofn));
-      }
-    };
-  }
-
-  private static final Map<Class<? extends PTransform>, TransformEvaluator<?>> EVALUATORS = Maps
-      .newHashMap();
-
-  static {
-    EVALUATORS.put(ConsoleIO.Write.Unbound.class, print());
-    EVALUATORS.put(CreateStream.QueuedValues.class, createFromQueue());
-    EVALUATORS.put(Create.Values.class, create());
-    EVALUATORS.put(KafkaIO.Read.Unbound.class, kafka());
-    EVALUATORS.put(Window.Bound.class, window());
-    EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
-  }
-
-  private static final Set<Class<? extends PTransform>> UNSUPPORTED_EVALUATORS = Sets
-      .newHashSet();
-
-  static {
-    //TODO - add support for the following
-    UNSUPPORTED_EVALUATORS.add(TextIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(TextIO.Write.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(AvroIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(AvroIO.Write.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(HadoopIO.Read.Bound.class);
-    UNSUPPORTED_EVALUATORS.add(HadoopIO.Write.Bound.class);
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <PT extends PTransform<?, ?>> TransformEvaluator<PT>
-      getTransformEvaluator(Class<PT> clazz, SparkPipelineTranslator rddTranslator) {
-    TransformEvaluator<PT> transform = (TransformEvaluator<PT>) EVALUATORS.get(clazz);
-    if (transform == null) {
-      if (UNSUPPORTED_EVALUATORS.contains(clazz)) {
-        throw new UnsupportedOperationException("Dataflow transformation " + clazz
-          .getCanonicalName()
-          + " is currently unsupported by the Spark streaming pipeline");
-      }
-      // DStream transformations will transform an RDD into another RDD
-      // Actions will create output
-      // In Dataflow it depends on the PTransform's Input and Output class
-      Class<?> pTOutputClazz = getPTransformOutputClazz(clazz);
-      if (PDone.class.equals(pTOutputClazz)) {
-        return foreachRDD(rddTranslator);
-      } else {
-        return rddTransform(rddTranslator);
-      }
-    }
-    return transform;
-  }
-
-  private static <PT extends PTransform<?, ?>> Class<?> getPTransformOutputClazz(Class<PT> clazz) {
-    Type[] types = ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments();
-    return TypeToken.of(clazz).resolveType(types[1]).getRawType();
-  }
-
-  /**
-   * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator.
-   * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation
-   */
-  public static class Translator implements SparkPipelineTranslator {
-
-    private final SparkPipelineTranslator rddTranslator;
-
-    public Translator(SparkPipelineTranslator rddTranslator) {
-      this.rddTranslator = rddTranslator;
-    }
-
-    @Override
-    public boolean hasTranslation(Class<? extends PTransform<?, ?>> clazz) {
-      // streaming includes rdd transformations as well
-      return EVALUATORS.containsKey(clazz) || rddTranslator.hasTranslation(clazz);
-    }
-
-    @Override
-    public <PT extends PTransform<?, ?>> TransformEvaluator<PT> translate(Class<PT> clazz) {
-      return getTransformEvaluator(clazz, rddTranslator);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java
deleted file mode 100644
index 406dfcc..0000000
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
- *
- * Cloudera, Inc. 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
- *
- * This software 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 com.cloudera.dataflow.spark.streaming;
-
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.Durations;
-
-import com.cloudera.dataflow.spark.SparkPipelineRunner;
-import com.cloudera.dataflow.spark.SparkPipelineTranslator;
-import com.cloudera.dataflow.spark.TransformTranslator;
-
-/**
- * Pipeline {@link SparkPipelineRunner.Evaluator} to detect windowing.
- */
-public final class StreamingWindowPipelineDetector extends SparkPipelineRunner.Evaluator {
-
-  // Currently, Spark streaming recommends batches no smaller then 500 msec
-  private static final Duration SPARK_MIN_WINDOW = Durations.milliseconds(500);
-
-  private boolean windowing;
-  private Duration batchDuration;
-
-  public StreamingWindowPipelineDetector(SparkPipelineTranslator translator) {
-    super(translator);
-  }
-
-  private static final TransformTranslator.FieldGetter WINDOW_FG =
-      new TransformTranslator.FieldGetter(Window.Bound.class);
-
-  // Use the smallest window (fixed or sliding) as Spark streaming's batch duration
-  @Override
-  protected <PT extends PTransform<? super PInput, POutput>> void
-      doVisitTransform(TransformTreeNode node) {
-    @SuppressWarnings("unchecked")
-    PT transform = (PT) node.getTransform();
-    @SuppressWarnings("unchecked")
-    Class<PT> transformClass = (Class<PT>) (Class<?>) transform.getClass();
-    if (transformClass.isAssignableFrom(Window.Bound.class)) {
-      WindowFn<?, ?> windowFn = WINDOW_FG.get("windowFn", transform);
-      if (windowFn instanceof FixedWindows) {
-        setBatchDuration(((FixedWindows) windowFn).getSize());
-      } else if (windowFn instanceof SlidingWindows) {
-        if (((SlidingWindows) windowFn).getOffset().getMillis() > 0) {
-          throw new UnsupportedOperationException("Spark does not support window offsets");
-        }
-        // Sliding window size might as well set the batch duration. Applying the transformation
-        // will add the "slide"
-        setBatchDuration(((SlidingWindows) windowFn).getSize());
-      } else if (!(windowFn instanceof GlobalWindows)) {
-        throw new IllegalStateException("Windowing function not supported: " + windowFn);
-      }
-    }
-  }
-
-  private void setBatchDuration(org.joda.time.Duration duration) {
-    Long durationMillis = duration.getMillis();
-    // validate window size
-    if (durationMillis < SPARK_MIN_WINDOW.milliseconds()) {
-      throw new IllegalArgumentException("Windowing of size " + durationMillis +
-          "msec is not supported!");
-    }
-    // choose the smallest duration to be Spark's batch duration, larger ones will be handled
-    // as window functions  over the batched-stream
-    if (!windowing || this.batchDuration.milliseconds() > durationMillis) {
-      this.batchDuration = Durations.milliseconds(durationMillis);
-    }
-    windowing = true;
-  }
-
-  public boolean isWindowing() {
-    return windowing;
-  }
-
-  public Duration getBatchDuration() {
-    return batchDuration;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/41c4ca6a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
new file mode 100644
index 0000000..6af829f
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/DoFnFunction.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved.
+ *
+ * Cloudera, Inc. 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
+ *
+ * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ * CONDITIONS OF ANY KIND, either express or implied. See the License for
+ * the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.spark.api.java.function.FlatMapFunction;
+
+/**
+ * Dataflow's Do functions correspond to Spark's FlatMap functions.
+ *
+ * @param <I> Input element type.
+ * @param <O> Output element type.
+ */
+public class DoFnFunction<I, O> implements FlatMapFunction<Iterator<WindowedValue<I>>,
+    WindowedValue<O>> {
+  private final DoFn<I, O> mFunction;
+  private final SparkRuntimeContext mRuntimeContext;
+  private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+
+  /**
+   * @param fn         DoFunction to be wrapped.
+   * @param runtime    Runtime to apply function in.
+   * @param sideInputs Side inputs used in DoFunction.
+   */
+  public DoFnFunction(DoFn<I, O> fn,
+               SparkRuntimeContext runtime,
+               Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+    this.mFunction = fn;
+    this.mRuntimeContext = runtime;
+    this.mSideInputs = sideInputs;
+  }
+
+  @Override
+  public Iterable<WindowedValue<O>> call(Iterator<WindowedValue<I>> iter) throws
+      Exception {
+    ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs);
+    ctxt.setup();
+    mFunction.startBundle(ctxt);
+    return ctxt.getOutputIterable(iter, mFunction);
+  }
+
+  private class ProcCtxt extends SparkProcessContext<I, O, WindowedValue<O>> {
+
+    private final List<WindowedValue<O>> outputs = new LinkedList<>();
+
+    ProcCtxt(DoFn<I, O> fn, SparkRuntimeContext runtimeContext, Map<TupleTag<?>,
+        BroadcastHelper<?>> sideInputs) {
+      super(fn, runtimeContext, sideInputs);
+    }
+
+    @Override
+    public synchronized void output(O o) {
+      outputs.add(windowedValue != null ? windowedValue.withValue(o) :
+          WindowedValue.valueInEmptyWindows(o));
+    }
+
+    @Override
+    public synchronized void output(WindowedValue<O> o) {
+      outputs.add(o);
+    }
+
+    @Override
+    protected void clearOutput() {
+      outputs.clear();
+    }
+
+    @Override
+    protected Iterator<WindowedValue<O>> getOutputIterator() {
+      return outputs.iterator();
+    }
+  }
+
+}