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();
+ }
+
+ /**
+ * => 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>-->
+ <!--<!– 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>-->
</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>-->
- <!--<!– 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>
+ <!-- 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();
- }
-
- /**
- * => 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();
+ }
+ }
+
+}