From ad444470e97f7a1a9bc4582831f9d6df334e9949 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Thu, 10 Jul 2014 18:59:48 -0700 Subject: [PATCH 001/137] Initial commit --- runners/spark/.gitignore | 10 +++++++ runners/spark/pom.xml | 58 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 68 insertions(+) create mode 100644 runners/spark/.gitignore create mode 100644 runners/spark/pom.xml diff --git a/runners/spark/.gitignore b/runners/spark/.gitignore new file mode 100644 index 000000000000..0b4510ca11dc --- /dev/null +++ b/runners/spark/.gitignore @@ -0,0 +1,10 @@ +.classpath +.project +.settings +.cache +target +*.iml +.idea +gen +.DS_Store +dependency-reduced-pom.xml \ No newline at end of file diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml new file mode 100644 index 000000000000..231dd6bec944 --- /dev/null +++ b/runners/spark/pom.xml @@ -0,0 +1,58 @@ + + + + 4.0.0 + com.cloudera.dataflow + spark-dataflow + 0.0.1-SNAPSHOT + An Apache Spark implementation of Google Cloud Dataflow + Spark Dataflow + jar + + + 1.0.140709 + 1.0.0 + + + + + com.google.cloud.dataflow + dataflow-sdk + ${dataflow.version} + + + + org.apache.spark + spark-core_2.10 + ${spark.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + org.apache.hadoop.streaming.HadoopStreaming + + + + + + + From 845a8178ab8bb620d0278b8312f04dce3f1348d8 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Thu, 10 Jul 2014 21:25:35 -0700 Subject: [PATCH 002/137] Dumbest proof of concept possible --- runners/spark/pom.xml | 175 ++++++++++---- .../cloudera/dataflow/spark/DoFnFunction.java | 88 +++++++ .../dataflow/spark/SparkPipelineRunner.java | 222 ++++++++++++++++++ .../dataflow/spark/WordCountTest.java | 84 +++++++ 4 files changed, 528 insertions(+), 41 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 231dd6bec944..88fa4376eb29 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -1,58 +1,151 @@ - - + + 4.0.0 - com.cloudera.dataflow - spark-dataflow - 0.0.1-SNAPSHOT - An Apache Spark implementation of Google Cloud Dataflow - Spark Dataflow + Cloud Dataflow Examples + com.google.cloud.dataflow + examples + 1 jar - 1.0.140709 - 1.0.0 + UTF-8 + + + dataflow-snapshot + file://${project.basedir}/jars + + true + + + true + always + + + + + + + + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + + org.apache.felix + maven-bundle-plugin + 2.4.0 + true + + + + *;scope=compile|runtime;inline=true + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.8 + + + copy-dependencies + package + + copy-dependencies + + + + + + + + + org.apache.spark + spark-core_2.10 + 1.0.0 + + com.google.cloud.dataflow dataflow-sdk - ${dataflow.version} + 1.0.140709 + + + com.google.apis + google-api-services-storage + v1-rev2-1.18.0-rc + + + com.google.apis + google-api-services-bigquery + v2-rev149-1.18.0-rc + + + com.google.http-client + google-http-client-jackson2 + 1.18.0-rc + + + com.google.oauth-client + google-oauth-client-jetty + 1.18.0-rc - org.apache.spark - spark-core_2.10 - ${spark.version} + com.fasterxml.jackson.core + jackson-core + 2.4.1 + + + com.fasterxml.jackson.core + jackson-annotations + 2.4.1 - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - org.apache.hadoop.streaming.HadoopStreaming - - - - - - + + + commons-cli + commons-cli + 1.2 + + + + + org.hamcrest + hamcrest-all + 1.3 + test + + + junit + junit + 4.11 + test + + 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 new file mode 100644 index 000000000000..d61bdd2088f0 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.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 com.cloudera.dataflow.spark; + +import com.google.api.client.util.Lists; +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.streaming.KeyedState; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.apache.spark.api.java.function.FlatMapFunction; + +import java.util.Iterator; +import java.util.List; + +public class DoFnFunction implements FlatMapFunction, O> { + + private final DoFn fn; + + public DoFnFunction(DoFn fn) { + this.fn = fn; + } + + @Override + public Iterable call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt(fn); + fn.startBatch(ctxt); + while (iter.hasNext()) { + ctxt.element = iter.next(); + fn.processElement(ctxt); + } + fn.finishBatch(ctxt); + return ctxt.outputs; + } + + private static class ProcCtxt extends DoFn.ProcessContext { + + private List outputs = Lists.newArrayList(); + private I element; + + public ProcCtxt(DoFn fn) { + fn.super(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return null; + } + + @Override + public void output(O o) { + outputs.add(o); + } + + @Override + public void sideOutput(TupleTag tupleTag, T t) { + } + + @Override + public T sideInput(TupleTag tupleTag) { + return null; + } + + @Override + public I element() { + return element; + } + + @Override + public KeyedState keyedState() { + return null; + } + } +} 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 new file mode 100644 index 000000000000..3b31dca62b6f --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -0,0 +1,222 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +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.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.PrimitivePTransform; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +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.PairFunction; +import scala.Tuple2; +import scala.collection.JavaConversions; + +import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.Map; + +public class SparkPipelineRunner extends PipelineRunner { + + @Override + public EvaluationResults run(Pipeline pipeline) { + EvaluationContext ctxt = new EvaluationContext(); + pipeline.traverseTopologically(new Evaluator(ctxt)); + return ctxt; + } + + private class Evaluator implements Pipeline.PipelineVisitor { + + private final EvaluationContext ctxt; + + private Evaluator(EvaluationContext ctxt) { + this.ctxt = ctxt; + } + + @Override + public void visitTransform(PrimitivePTransform primitive) { + EVALUATORS.get(primitive.getClass()).evaluate(primitive, ctxt); + } + + @Override + public void visitValue(PValue value) { + System.out.println("VV: " + value); + } + } + + public static interface EvaluationResults { + + } + + public static class EvaluationContext implements EvaluationResults { + JavaSparkContext jsc = new JavaSparkContext("local", "dataflow"); + JavaRDDLike last; + + JavaSparkContext getSparkContext() { + return jsc; + } + void setLast(JavaRDDLike rdd) { + last = rdd; + } + + JavaRDDLike getLast() { return last; } + } + + public static interface TransformEvaluator extends Serializable { + void evaluate(PT transform, EvaluationContext context); + } + + private static class FieldGetter { + private Map fields; + + public FieldGetter(Class clazz) { + this.fields = Maps.newHashMap(); + for (Field f : clazz.getDeclaredFields()) { + try { + f.setAccessible(true); + this.fields.put(f.getName(), f); + System.err.println("Field " + f.getName() + " for class = " + clazz); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + public T get(String fieldname, Object value) { + try { + return (T) fields.get(fieldname).get(value); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + + private static FieldGetter READ_TEXT_FG = new FieldGetter(TextIO.Read.Bound.class); + private static FieldGetter WRITE_TEXT_FG = new FieldGetter(TextIO.Write.Bound.class); + + private static TransformEvaluator READ_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { + String pattern = READ_TEXT_FG.get("filepattern", transform); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + Coder coder = READ_TEXT_FG.get("coder", transform); + if (coder != null) { + //TODO + } + context.setLast(rdd); + } + }; + + private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getLast(); + Coder coder = WRITE_TEXT_FG.get("coder", transform); + if (coder != null) { + //TODO + } + String pattern = WRITE_TEXT_FG.get("filepattern", transform); + last.saveAsTextFile(pattern); + } + }; + + private static FieldGetter CREATE_FG = new FieldGetter(Create.class); + private static TransformEvaluator CREATE = new TransformEvaluator() { + @Override + public void evaluate(Create transform, EvaluationContext context) { + Iterable elems = CREATE_FG.get("elems", transform); + JavaRDD rdd = context.getSparkContext().parallelize(Lists.newLinkedList(elems)); + context.setLast(rdd); + } + }; + + private static FieldGetter PARDO_FG = new FieldGetter(ParDo.Bound.class); + private static TransformEvaluator PARDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getLast(); + DoFnFunction dofn = new DoFnFunction(PARDO_FG.get("fn", transform)); + context.setLast(last.mapPartitions(dofn)); + } + }; + + private static TransformEvaluator GBK = new TransformEvaluator() { + @Override + public void evaluate(GroupByKey transform, EvaluationContext context) { + context.setLast(fromPair(toPair(context.getLast()).groupByKey())); + } + + private JavaPairRDD toPair(JavaRDDLike rdd) { + return rdd.mapToPair(new PairFunction() { + @Override + public Tuple2 call(Object o) throws Exception { + KV kv = (KV) o; + return new Tuple2(kv.getKey(), kv.getValue()); + } + }); + } + + private JavaRDDLike fromPair(JavaPairRDD rdd) { + return rdd.map(new Function() { + @Override + public Object call(Object o) throws Exception { + Tuple2 t2 = (Tuple2) o; + return KV.of(t2._1(), t2._2()); + } + }); + } + }; + + private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); + private static TransformEvaluator GROUPED = new TransformEvaluator() { + @Override + public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { + final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + context.setLast(context.getLast().map(new Function() { + @Override + public Object call(Object input) throws Exception { + KV kv = (KV) input; + return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); + } + })); + } + }; + + private static final Map EVALUATORS = ImmutableMap.builder() + .put(Combine.GroupedValues.class, GROUPED) + .put(GroupByKey.class, GBK) + .put(ParDo.Bound.class, PARDO) + .put(TextIO.Read.Bound.class, READ_TEXT) + .put(TextIO.Write.Bound.class, WRITE_TEXT) + .put(Create.class, CREATE) + .build(); +} diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java new file mode 100644 index 000000000000..b13534f84fc9 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Count; +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.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import org.junit.Test; + +public class WordCountTest { + + /** A DoFn that tokenizes lines of text into individual words. */ + static class ExtractWordsFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + String[] words = c.element().split("[^a-zA-Z']+"); + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + /** A DoFn that converts a Word and Count into a printable string. */ + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey() + ": " + c.element().getValue()); + } + } + + public static class CountWords extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + PCollection> wordCounts = + words.apply(Count.create()); + + // Format each word and count into a printable string. + PCollection results = wordCounts.apply( + ParDo.of(new FormatCountsFn())); + + return results; + } + } + + @Test + public void testRun() throws Exception { + Pipeline p = Pipeline.create(); + + p.apply(TextIO.Read.named("ReadLines").from("/tmp/test.txt")) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts").to("/tmp/dfout.txt")); + + SparkPipelineRunner.EvaluationResults results = p.run(new SparkPipelineRunner()); + System.out.println(results); + } +} From cb7c86642ea9839e3638c16849e113cb3536604a Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 3 Nov 2014 16:40:14 -0800 Subject: [PATCH 003/137] First bit of work to get this running against the new Dataflow API --- runners/spark/pom.xml | 28 +++++---- .../cloudera/dataflow/spark/DoFnFunction.java | 17 +++++ .../dataflow/spark/SparkPipeline.java | 37 +++++++++++ .../dataflow/spark/SparkPipelineRunner.java | 63 ++++++++++++++----- .../dataflow/spark/WordCountTest.java | 7 ++- 5 files changed, 122 insertions(+), 30 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 88fa4376eb29..77560fbea3b8 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -87,44 +87,48 @@ org.apache.spark spark-core_2.10 - 1.0.0 + 1.1.0 com.google.cloud.dataflow dataflow-sdk - 1.0.140709 + 1.0.141027 com.google.apis google-api-services-storage - v1-rev2-1.18.0-rc + v1-rev11-1.19.0 com.google.apis google-api-services-bigquery - v2-rev149-1.18.0-rc + v2-rev167-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.http-client google-http-client-jackson2 - 1.18.0-rc - - - com.google.oauth-client - google-oauth-client-jetty - 1.18.0-rc + 1.19.0 com.fasterxml.jackson.core jackson-core - 2.4.1 + 2.4.2 com.fasterxml.jackson.core jackson-annotations - 2.4.1 + 2.4.2 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 index d61bdd2088f0..8c1efcc396f4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -20,7 +20,10 @@ import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.runners.PipelineOptions; import com.google.cloud.dataflow.sdk.streaming.KeyedState; +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.SerializableFunction; import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.spark.api.java.function.FlatMapFunction; @@ -70,6 +73,20 @@ public void output(O o) { public void sideOutput(TupleTag tupleTag, T t) { } + @Override + public Aggregator createAggregator( + String s, + Combine.CombineFn combineFn) { + return null; + } + + @Override + public Aggregator createAggregator( + String s, + SerializableFunction, AO> sfunc) { + return null; + } + @Override public T sideInput(TupleTag tupleTag) { return null; diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java new file mode 100644 index 000000000000..f20f1b1a7473 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; + +public class SparkPipeline extends Pipeline { + + public static SparkPipeline create(String master) { + return new SparkPipeline(new SparkPipelineRunner(master)); + } + + protected SparkPipeline(PipelineRunner runner) { + super(runner); + } + + protected SparkPipeline(PipelineRunner runner, PipelineOptions options) { + super(runner, options); + } +} 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 index 3b31dca62b6f..e15b9a6d26b5 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -17,17 +17,20 @@ */ package com.cloudera.dataflow.spark; +import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; 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.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.transforms.PrimitivePTransform; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.common.collect.ImmutableMap; @@ -39,17 +42,30 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; -import scala.collection.JavaConversions; import java.io.Serializable; import java.lang.reflect.Field; import java.util.Map; +import java.util.logging.Logger; -public class SparkPipelineRunner extends PipelineRunner { +public class SparkPipelineRunner extends PipelineRunner { + + private static final Logger LOG = + Logger.getLogger(SparkPipelineRunner.class.getName()); + + private final String master; + + public SparkPipelineRunner() { + this("local"); + } + + public SparkPipelineRunner(String master) { + this.master = Preconditions.checkNotNull(master); + } @Override - public EvaluationResults run(Pipeline pipeline) { - EvaluationContext ctxt = new EvaluationContext(); + public EvaluationResult run(Pipeline pipeline) { + EvaluationContext ctxt = new EvaluationContext(this.master); pipeline.traverseTopologically(new Evaluator(ctxt)); return ctxt; } @@ -63,24 +79,41 @@ private Evaluator(EvaluationContext ctxt) { } @Override - public void visitTransform(PrimitivePTransform primitive) { - EVALUATORS.get(primitive.getClass()).evaluate(primitive, ctxt); + public void enterCompositeTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value) { - System.out.println("VV: " + value); + public void leaveCompositeTransform(TransformTreeNode node) { + } + + @Override + public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + TransformEvaluator evaluator = EVALUATORS.get(transform.getClass()); + if (evaluator == null) { + throw new IllegalStateException( + "no evaluator registered for " + transform); + } + LOG.info("Evaluating " + transform); + evaluator.evaluate(transform, ctxt); } - } - public static interface EvaluationResults { + @Override + public void visitValue(PValue pvalue, TransformTreeNode node) { + } + } + public static interface EvaluationResult extends PipelineResult { } - public static class EvaluationContext implements EvaluationResults { - JavaSparkContext jsc = new JavaSparkContext("local", "dataflow"); + public static class EvaluationContext implements EvaluationResult { + final JavaSparkContext jsc; JavaRDDLike last; + public EvaluationContext(String master) { + this.jsc = new JavaSparkContext(master, "dataflow"); + } + JavaSparkContext getSparkContext() { return jsc; } @@ -91,7 +124,7 @@ void setLast(JavaRDDLike rdd) { JavaRDDLike getLast() { return last; } } - public static interface TransformEvaluator extends Serializable { + public static interface TransformEvaluator extends Serializable { void evaluate(PT transform, EvaluationContext context); } @@ -144,7 +177,7 @@ public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { if (coder != null) { //TODO } - String pattern = WRITE_TEXT_FG.get("filepattern", transform); + String pattern = WRITE_TEXT_FG.get("filenamePrefix", transform); last.saveAsTextFile(pattern); } }; diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index b13534f84fc9..f118dcbadfa6 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -18,6 +18,7 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.DoFn; @@ -72,13 +73,13 @@ public PCollection apply(PCollection lines) { @Test public void testRun() throws Exception { - Pipeline p = Pipeline.create(); + SparkPipeline p = SparkPipeline.create("local"); p.apply(TextIO.Read.named("ReadLines").from("/tmp/test.txt")) .apply(new CountWords()) .apply(TextIO.Write.named("WriteCounts").to("/tmp/dfout.txt")); - SparkPipelineRunner.EvaluationResults results = p.run(new SparkPipelineRunner()); - System.out.println(results); + PipelineResult result = p.run(); + System.out.println(result); } } From dce03e4b7d607bf8ac9846fa4bfb26a81d747a63 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 1 Dec 2014 17:58:30 -0800 Subject: [PATCH 004/137] Update version of dataflow to get new API method access --- runners/spark/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 77560fbea3b8..1ac764b5873e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -93,7 +93,7 @@ com.google.cloud.dataflow dataflow-sdk - 1.0.141027 + 0.0.1 com.google.apis From 08e94b28ea06667049817def8aa5aee7aa31ebe1 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 1 Dec 2014 18:23:05 -0800 Subject: [PATCH 005/137] Add support for getters and a Flatten impl --- .../cloudera/dataflow/spark/DoFnFunction.java | 12 +-- .../dataflow/spark/SparkPipelineRunner.java | 102 +++++++++--------- .../dataflow/spark/WordCountTest.java | 1 - 3 files changed, 54 insertions(+), 61 deletions(-) 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 index 8c1efcc396f4..97c21078ad32 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -17,7 +17,6 @@ */ package com.cloudera.dataflow.spark; -import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.runners.PipelineOptions; import com.google.cloud.dataflow.sdk.streaming.KeyedState; import com.google.cloud.dataflow.sdk.transforms.Aggregator; @@ -28,6 +27,7 @@ import org.apache.spark.api.java.function.FlatMapFunction; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; public class DoFnFunction implements FlatMapFunction, O> { @@ -50,9 +50,9 @@ public Iterable call(Iterator iter) throws Exception { return ctxt.outputs; } - private static class ProcCtxt extends DoFn.ProcessContext { + private class ProcCtxt extends DoFn.ProcessContext { - private List outputs = Lists.newArrayList(); + private List outputs = new LinkedList<>(); private I element; public ProcCtxt(DoFn fn) { @@ -65,7 +65,7 @@ public PipelineOptions getPipelineOptions() { } @Override - public void output(O o) { + public synchronized void output(O o) { outputs.add(o); } @@ -75,14 +75,14 @@ public void sideOutput(TupleTag tupleTag, T t) { @Override public Aggregator createAggregator( - String s, + String named, Combine.CombineFn combineFn) { return null; } @Override public Aggregator createAggregator( - String s, + String named, SerializableFunction, AO> sfunc) { return null; } 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 index e15b9a6d26b5..cb8e1f4d3785 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -28,10 +28,14 @@ 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.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.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -65,7 +69,7 @@ public SparkPipelineRunner(String master) { @Override public EvaluationResult run(Pipeline pipeline) { - EvaluationContext ctxt = new EvaluationContext(this.master); + EvaluationContext ctxt = new EvaluationContext(this.master, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); return ctxt; } @@ -108,104 +112,81 @@ public static interface EvaluationResult extends PipelineResult { public static class EvaluationContext implements EvaluationResult { final JavaSparkContext jsc; - JavaRDDLike last; + final Pipeline pipeline; + final Map rdds = Maps.newHashMap(); - public EvaluationContext(String master) { + public EvaluationContext(String master, Pipeline pipeline) { this.jsc = new JavaSparkContext(master, "dataflow"); + this.pipeline = pipeline; } JavaSparkContext getSparkContext() { return jsc; } - void setLast(JavaRDDLike rdd) { - last = rdd; - } - - JavaRDDLike getLast() { return last; } - } + Pipeline getPipeline() { return pipeline; } - public static interface TransformEvaluator extends Serializable { - void evaluate(PT transform, EvaluationContext context); - } + void setRDD(PTransform transform, JavaRDDLike rdd) { + rdds.put((PValue) pipeline.getOutput(transform), rdd); + } - private static class FieldGetter { - private Map fields; - - public FieldGetter(Class clazz) { - this.fields = Maps.newHashMap(); - for (Field f : clazz.getDeclaredFields()) { - try { - f.setAccessible(true); - this.fields.put(f.getName(), f); - System.err.println("Field " + f.getName() + " for class = " + clazz); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + JavaRDDLike getRDD(PCollection pcollection) { + return rdds.get(pcollection); } - public T get(String fieldname, Object value) { - try { - return (T) fields.get(fieldname).get(value); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } + JavaRDDLike getRDD(PTransform transform) { + return rdds.get(pipeline.getInput(transform)); } } - private static FieldGetter READ_TEXT_FG = new FieldGetter(TextIO.Read.Bound.class); - private static FieldGetter WRITE_TEXT_FG = new FieldGetter(TextIO.Write.Bound.class); + public static interface TransformEvaluator extends Serializable { + void evaluate(PT transform, EvaluationContext context); + } private static TransformEvaluator READ_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { - String pattern = READ_TEXT_FG.get("filepattern", transform); + String pattern = transform.getFilepattern(); JavaRDD rdd = context.getSparkContext().textFile(pattern); - Coder coder = READ_TEXT_FG.get("coder", transform); - if (coder != null) { - //TODO - } - context.setLast(rdd); + // TODO: handle coders + context.setRDD(transform, rdd); } }; private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getLast(); - Coder coder = WRITE_TEXT_FG.get("coder", transform); + JavaRDDLike last = context.getRDD(transform); + Coder coder = null; if (coder != null) { //TODO } - String pattern = WRITE_TEXT_FG.get("filenamePrefix", transform); + String pattern = transform.getFilenamePrefix(); last.saveAsTextFile(pattern); } }; - private static FieldGetter CREATE_FG = new FieldGetter(Create.class); private static TransformEvaluator CREATE = new TransformEvaluator() { @Override public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = CREATE_FG.get("elems", transform); + Iterable elems = transform.getElems(); JavaRDD rdd = context.getSparkContext().parallelize(Lists.newLinkedList(elems)); - context.setLast(rdd); + context.setRDD(transform, rdd); } }; - private static FieldGetter PARDO_FG = new FieldGetter(ParDo.Bound.class); private static TransformEvaluator PARDO = new TransformEvaluator() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getLast(); - DoFnFunction dofn = new DoFnFunction(PARDO_FG.get("fn", transform)); - context.setLast(last.mapPartitions(dofn)); + JavaRDDLike last = context.getRDD(transform); + DoFnFunction dofn = new DoFnFunction(transform.getFn()); + context.setRDD(transform, last.mapPartitions(dofn)); } }; private static TransformEvaluator GBK = new TransformEvaluator() { @Override public void evaluate(GroupByKey transform, EvaluationContext context) { - context.setLast(fromPair(toPair(context.getLast()).groupByKey())); + context.setRDD(transform, fromPair(toPair(context.getRDD(transform)).groupByKey())); } private JavaPairRDD toPair(JavaRDDLike rdd) { @@ -229,12 +210,11 @@ public Object call(Object o) throws Exception { } }; - private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); private static TransformEvaluator GROUPED = new TransformEvaluator() { @Override public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); - context.setLast(context.getLast().map(new Function() { + final Combine.KeyedCombineFn keyed = transform.getFn(); + context.setRDD(transform, context.getRDD(transform).map(new Function() { @Override public Object call(Object input) throws Exception { KV kv = (KV) input; @@ -244,6 +224,19 @@ public Object call(Object input) throws Exception { } }; + private static TransformEvaluator FLATTEN = new TransformEvaluator() { + @Override + public void evaluate(Flatten transform, EvaluationContext context) { + PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + JavaRDD[] rdds = new JavaRDD[pcs.size()]; + for (int i = 0; i < rdds.length; i++) { + rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + } + JavaRDD rdd = context.getSparkContext().union(rdds); + context.setRDD(transform, rdd); + } + }; + private static final Map EVALUATORS = ImmutableMap.builder() .put(Combine.GroupedValues.class, GROUPED) .put(GroupByKey.class, GBK) @@ -251,5 +244,6 @@ public Object call(Object input) throws Exception { .put(TextIO.Read.Bound.class, READ_TEXT) .put(TextIO.Write.Bound.class, WRITE_TEXT) .put(Create.class, CREATE) + .put(Flatten.class, FLATTEN) .build(); } diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index f118dcbadfa6..26c340b6bdab 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -17,7 +17,6 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Count; From 9fdac6ccb750488cdff8e3a696d114911d385ee4 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 1 Dec 2014 22:54:28 -0800 Subject: [PATCH 006/137] Such code. Much features. --- .../dataflow/spark/BroadcastHelper.java | 52 ++++++ .../cloudera/dataflow/spark/CoderHelpers.java | 77 ++++++++ .../dataflow/spark/CoderSparkSerializer.java | 67 +++++++ .../cloudera/dataflow/spark/DoFnFunction.java | 16 +- .../dataflow/spark/EvaluationContext.java | 98 ++++++++++ .../dataflow/spark/SparkPipelineRunner.java | 175 ++++++++++-------- .../dataflow/spark/TransformEvaluator.java | 26 +++ 7 files changed, 429 insertions(+), 82 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.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 new file mode 100644 index 000000000000..670bbc8de5eb --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import org.apache.spark.broadcast.Broadcast; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.Serializable; + +class BroadcastHelper implements Serializable { + private final Broadcast bcast; + private final Coder coder; + private transient T value; + + BroadcastHelper(Broadcast bcast, Coder coder) { + this.bcast = bcast; + this.coder = coder; + } + + public synchronized T getValue() { + if (value == null) { + value = deserialize(); + } + return value; + } + + private T deserialize() { + try { + coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error deserializing broadcast variable", e); + } + return null; + } +} 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 new file mode 100644 index 000000000000..3e7bd1b20974 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.spark.api.java.function.Function; + +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collection; +import java.util.List; + +public class CoderHelpers { + static byte[] toByteArray(Object value, Coder coder) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + coder.encode(value, baos, new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error encoding value: " + value, e); + } + return baos.toByteArray(); + } + + static List toByteArrays(Iterable values, final Coder coder) { + List res = Lists.newLinkedList(); + for (Object value : values) { + res.add(toByteArray(value, coder)); + } + return res; + } + + static T fromByteArray(byte[] serialized, Coder coder) { + ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + try { + return coder.decode(bais, new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error decoding bytes for coder: " + coder, e); + } + } + + static Function toByteFunction(final Coder coder) { + return new Function() { + @Override + public byte[] call(T t) throws Exception { + return toByteArray(t, coder); + } + }; + } + + static Function fromByteFunction(final Coder coder) { + return new Function() { + @Override + public T call(byte[] bytes) throws Exception { + return fromByteArray(bytes, coder); + } + }; + } +} diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java new file mode 100644 index 000000000000..3430d564f4d2 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import org.apache.spark.serializer.DeserializationStream; +import org.apache.spark.serializer.SerializationStream; +import org.apache.spark.serializer.Serializer; +import org.apache.spark.serializer.SerializerInstance; +import scala.reflect.ClassTag; + +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; + +public class CoderSparkSerializer extends Serializer implements Serializable { + + @Override + public SerializerInstance newInstance() { + return new CoderSerializerInstance(); + } + + private class CoderSerializerInstance extends SerializerInstance implements Serializable { + + @Override + public ByteBuffer serialize(T t, ClassTag ev) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassTag ev) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassLoader loader, ClassTag ev) { + return null; + } + + @Override + public SerializationStream serializeStream(OutputStream os) { + return null; + } + + @Override + public DeserializationStream deserializeStream(InputStream is) { + return null; + } + } +} 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 index 97c21078ad32..8bdeae4d682e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -24,18 +24,26 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.ImmutableMap; import org.apache.spark.api.java.function.FlatMapFunction; import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Map; public class DoFnFunction implements FlatMapFunction, O> { private final DoFn fn; + private final Map, BroadcastHelper> sideInputs; public DoFnFunction(DoFn fn) { + this(fn, ImmutableMap., BroadcastHelper>of()); + } + + public DoFnFunction(DoFn fn, Map, BroadcastHelper> sideInputs) { this.fn = fn; + this.sideInputs = sideInputs; } @Override @@ -71,6 +79,7 @@ public synchronized void output(O o) { @Override public void sideOutput(TupleTag tupleTag, T t) { + // A no-op if we don't know about it ahead of time } @Override @@ -88,8 +97,9 @@ public Aggregator createAggregator( } @Override - public T sideInput(TupleTag tupleTag) { - return null; + public T sideInput(TupleTag tag) { + BroadcastHelper bh = (BroadcastHelper) sideInputs.get(tag); + return bh == null ? null : bh.getValue(); } @Override @@ -99,7 +109,7 @@ public I element() { @Override public KeyedState keyedState() { - return null; + throw new UnsupportedOperationException(); } } } 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 new file mode 100644 index 000000000000..b12a216e6d24 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PObject; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.spark.api.java.JavaRDDLike; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; + +import java.util.Map; +import java.util.Set; + +public class EvaluationContext implements SparkPipelineRunner.EvaluationResult { + final JavaSparkContext jsc; + final Pipeline pipeline; + final Map rdds = Maps.newHashMap(); + final Set multireads = Sets.newHashSet(); + final Map localPObjects = Maps.newHashMap(); + + public EvaluationContext(String master, Pipeline pipeline) { + this.jsc = new JavaSparkContext(master, "dataflow"); + this.pipeline = pipeline; + } + + JavaSparkContext getSparkContext() { + return jsc; + } + Pipeline getPipeline() { return pipeline; } + + POutput getOutput(PTransform transform) { + return pipeline.getOutput(transform); + } + + void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + rdds.put((PValue) getOutput(transform), rdd); + } + + void setPObjectValue(PObject pobj, Object value) { + localPObjects.put(pobj, value); + } + + JavaRDDLike getRDD(PValue pvalue) { + JavaRDDLike rdd = rdds.get(pvalue); + if (multireads.contains(pvalue)) { + // Ensure the RDD is marked as cached + rdd.rdd().cache(); + } else { + multireads.add(pvalue); + } + return rdd; + } + + JavaRDDLike getInputRDD(PTransform transform) { + return getRDD((PValue) pipeline.getInput(transform)); + } + + BroadcastHelper getBroadcastHelper(PObject value) { + Coder coder = value.getCoder(); + Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(resolve(value), coder)); + return new BroadcastHelper<>(bcast, coder); + } + + T resolve(PObject value) { + if (localPObjects.containsKey(value)) { + return (T) localPObjects.get(value); + } else if (rdds.containsKey(value)) { + JavaRDDLike rdd = rdds.get(value); + //TODO: probably some work to do here + T res = (T) Iterables.getOnlyElement(rdd.collect()); + localPObjects.put(value, res); + return res; + } + throw new IllegalStateException("Cannot resolve un-known PObject: " + value); + } +} 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 index cb8e1f4d3785..9c8a4ef24b5c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -17,17 +17,17 @@ */ package com.cloudera.dataflow.spark; -import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; 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.CreatePObject; import com.google.cloud.dataflow.sdk.transforms.Flatten; import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; @@ -35,20 +35,20 @@ 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.POutput; +import com.google.cloud.dataflow.sdk.values.PObject; +import com.google.cloud.dataflow.sdk.values.PObjectTuple; import com.google.cloud.dataflow.sdk.values.PValue; -import com.google.common.collect.ImmutableMap; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TypedPValue; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; 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.PairFunction; import scala.Tuple2; -import java.io.Serializable; -import java.lang.reflect.Field; import java.util.Map; import java.util.logging.Logger; @@ -110,52 +110,40 @@ public void visitValue(PValue pvalue, TransformTreeNode node) { public static interface EvaluationResult extends PipelineResult { } - public static class EvaluationContext implements EvaluationResult { - final JavaSparkContext jsc; - final Pipeline pipeline; - final Map rdds = Maps.newHashMap(); - - public EvaluationContext(String master, Pipeline pipeline) { - this.jsc = new JavaSparkContext(master, "dataflow"); - this.pipeline = pipeline; - } - - JavaSparkContext getSparkContext() { - return jsc; - } - Pipeline getPipeline() { return pipeline; } - - void setRDD(PTransform transform, JavaRDDLike rdd) { - rdds.put((PValue) pipeline.getOutput(transform), rdd); - } - - JavaRDDLike getRDD(PCollection pcollection) { - return rdds.get(pcollection); - } - - JavaRDDLike getRDD(PTransform transform) { - return rdds.get(pipeline.getInput(transform)); - } - } - - public static interface TransformEvaluator extends Serializable { - void evaluate(PT transform, EvaluationContext context); - } - private static TransformEvaluator READ_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); JavaRDD rdd = context.getSparkContext().textFile(pattern); // TODO: handle coders - context.setRDD(transform, rdd); + context.setOutputRDD(transform, rdd); } }; private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getRDD(transform); + JavaRDDLike last = context.getInputRDD(transform); + // TODO: handle coders + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + + private static TransformEvaluator READ_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + // TODO: handle coders + context.setOutputRDD(transform, rdd); + } + }; + + private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); Coder coder = null; if (coder != null) { //TODO @@ -168,45 +156,64 @@ public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { private static TransformEvaluator CREATE = new TransformEvaluator() { @Override public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = transform.getElems(); - JavaRDD rdd = context.getSparkContext().parallelize(Lists.newLinkedList(elems)); - context.setRDD(transform, rdd); + Iterable elems = transform.getElements(); + Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); + JavaRDD rdd = context.getSparkContext().parallelize( + CoderHelpers.toByteArrays(elems, coder)); + context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + + private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { + @Override + public void evaluate(CreatePObject transform, EvaluationContext context) { + context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); } }; private static TransformEvaluator PARDO = new TransformEvaluator() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getRDD(transform); - DoFnFunction dofn = new DoFnFunction(transform.getFn()); - context.setRDD(transform, last.mapPartitions(dofn)); + JavaRDDLike last = context.getInputRDD(transform); + PObjectTuple pot = transform.getSideInputs(); + DoFnFunction dofn; + if (pot == null || pot.getAll().isEmpty()) { + dofn = new DoFnFunction(transform.getFn()); + } else { + Map, BroadcastHelper> sideInputs = Maps.newHashMap(); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); + } + dofn = new DoFnFunction(transform.getFn(), sideInputs); + } + context.setOutputRDD(transform, last.mapPartitions(dofn)); } }; + private static JavaPairRDD toPair(JavaRDDLike rdd) { + return rdd.mapToPair(new PairFunction() { + @Override + public Tuple2 call(Object o) throws Exception { + KV kv = (KV) o; + return new Tuple2(kv.getKey(), kv.getValue()); + } + }); + } + + private static JavaRDDLike fromPair(JavaPairRDD rdd) { + return rdd.map(new Function() { + @Override + public Object call(Object o) throws Exception { + Tuple2 t2 = (Tuple2) o; + return KV.of(t2._1(), t2._2()); + } + }); + } + private static TransformEvaluator GBK = new TransformEvaluator() { @Override public void evaluate(GroupByKey transform, EvaluationContext context) { - context.setRDD(transform, fromPair(toPair(context.getRDD(transform)).groupByKey())); - } - - private JavaPairRDD toPair(JavaRDDLike rdd) { - return rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Object o) throws Exception { - KV kv = (KV) o; - return new Tuple2(kv.getKey(), kv.getValue()); - } - }); - } - - private JavaRDDLike fromPair(JavaPairRDD rdd) { - return rdd.map(new Function() { - @Override - public Object call(Object o) throws Exception { - Tuple2 t2 = (Tuple2) o; - return KV.of(t2._1(), t2._2()); - } - }); + context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); } }; @@ -214,7 +221,7 @@ public Object call(Object o) throws Exception { @Override public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { final Combine.KeyedCombineFn keyed = transform.getFn(); - context.setRDD(transform, context.getRDD(transform).map(new Function() { + context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { @Override public Object call(Object input) throws Exception { KV kv = (KV) input; @@ -233,17 +240,27 @@ public void evaluate(Flatten transform, EvaluationContext context) { rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); } JavaRDD rdd = context.getSparkContext().union(rdds); - context.setRDD(transform, rdd); + context.setOutputRDD(transform, rdd); } }; - private static final Map EVALUATORS = ImmutableMap.builder() - .put(Combine.GroupedValues.class, GROUPED) - .put(GroupByKey.class, GBK) - .put(ParDo.Bound.class, PARDO) - .put(TextIO.Read.Bound.class, READ_TEXT) - .put(TextIO.Write.Bound.class, WRITE_TEXT) - .put(Create.class, CREATE) - .put(Flatten.class, FLATTEN) - .build(); + public static void registerEvaluator( + Class transformClass, + TransformEvaluator evaluator) { + EVALUATORS.put(transformClass, evaluator); + } + + private static final Map, TransformEvaluator> EVALUATORS = Maps.newHashMap(); + static { + registerEvaluator(TextIO.Read.Bound.class, READ_TEXT); + registerEvaluator(TextIO.Write.Bound.class, WRITE_TEXT); + registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); + registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); + registerEvaluator(ParDo.Bound.class, PARDO); + registerEvaluator(GroupByKey.class, GBK); + registerEvaluator(Combine.GroupedValues.class, GROUPED); + registerEvaluator(Flatten.class, FLATTEN); + registerEvaluator(Create.class, CREATE); + registerEvaluator(CreatePObject.class, CREATE_POBJ); + } } 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 new file mode 100644 index 000000000000..3147a1be7c65 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +import java.io.Serializable; + +public interface TransformEvaluator extends Serializable { + void evaluate(PT transform, EvaluationContext context); +} From deca2c0bdbe699099717591fc89afa4175c5438c Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 15:28:15 -0800 Subject: [PATCH 007/137] Adding some more operators: toiterable, seqdo --- .../dataflow/spark/EvaluationContext.java | 38 +++++++++++++-- ...arkPipeline.java => EvaluationResult.java} | 22 +++------ .../dataflow/spark/SparkPipelineRunner.java | 31 +++++++++--- .../dataflow/spark/WordCountTest.java | 47 ++++++++----------- 4 files changed, 85 insertions(+), 53 deletions(-) rename runners/spark/src/main/java/com/cloudera/dataflow/spark/{SparkPipeline.java => EvaluationResult.java} (61%) 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 index b12a216e6d24..bd6ec952fde4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -20,9 +20,15 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.SeqDo; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.PObject; +import com.google.cloud.dataflow.sdk.values.PObjectTuple; +import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -33,7 +39,7 @@ import java.util.Map; import java.util.Set; -public class EvaluationContext implements SparkPipelineRunner.EvaluationResult { +public class EvaluationContext implements EvaluationResult { final JavaSparkContext jsc; final Pipeline pipeline; final Map rdds = Maps.newHashMap(); @@ -50,6 +56,10 @@ JavaSparkContext getSparkContext() { } Pipeline getPipeline() { return pipeline; } + PInput getInput(PTransform transform) { + return pipeline.getInput(transform); + } + POutput getOutput(PTransform transform) { return pipeline.getOutput(transform); } @@ -79,11 +89,12 @@ JavaRDDLike getInputRDD(PTransform transform) { BroadcastHelper getBroadcastHelper(PObject value) { Coder coder = value.getCoder(); - Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(resolve(value), coder)); + Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(get(value), coder)); return new BroadcastHelper<>(bcast, coder); } - T resolve(PObject value) { + @Override + public T get(PObject value) { if (localPObjects.containsKey(value)) { return (T) localPObjects.get(value); } else if (rdds.containsKey(value)) { @@ -95,4 +106,25 @@ T resolve(PObject value) { } throw new IllegalStateException("Cannot resolve un-known PObject: " + value); } + + @Override + public Iterable get(PCollection pcollection) { + return getRDD(pcollection).collect(); + } + + PObjectValueTuple getPObjectTuple(PTransform transform) { + PObjectTuple pot = (PObjectTuple) pipeline.getInput(transform); + PObjectValueTuple povt = PObjectValueTuple.empty(); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + povt = povt.and((TupleTag) e.getKey(), get(e.getValue())); + } + return povt; + } + + void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { + PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + setPObjectValue(e.getValue(), outputValues.get(e.getKey())); + } + } } diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java similarity index 61% rename from runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java rename to runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java index f20f1b1a7473..94a50df0100b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipeline.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -17,21 +17,11 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; -import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PObject; -public class SparkPipeline extends Pipeline { - - public static SparkPipeline create(String master) { - return new SparkPipeline(new SparkPipelineRunner(master)); - } - - protected SparkPipeline(PipelineRunner runner) { - super(runner); - } - - protected SparkPipeline(PipelineRunner runner, PipelineOptions options) { - super(runner, options); - } +public interface EvaluationResult extends PipelineResult { + Iterable get(PCollection pcollection); + T get(PObject pobject); } 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 index 9c8a4ef24b5c..5d7a835247d3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -19,29 +19,29 @@ import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Convert; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.CreatePObject; 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.SeqDo; 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.PObject; import com.google.cloud.dataflow.sdk.values.PObjectTuple; +import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; -import com.google.cloud.dataflow.sdk.values.TypedPValue; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; @@ -52,7 +52,7 @@ import java.util.Map; import java.util.logging.Logger; -public class SparkPipelineRunner extends PipelineRunner { +public class SparkPipelineRunner extends PipelineRunner { private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); @@ -107,9 +107,6 @@ public void visitValue(PValue pvalue, TransformTreeNode node) { } } - public static interface EvaluationResult extends PipelineResult { - } - private static TransformEvaluator READ_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { @@ -171,6 +168,15 @@ public void evaluate(CreatePObject transform, EvaluationContext context) { } }; + private static TransformEvaluator TO_ITER = new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterable transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, context.get(in)); + } + }; + private static TransformEvaluator PARDO = new TransformEvaluator() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { @@ -210,6 +216,15 @@ public Object call(Object o) throws Exception { }); } + private static TransformEvaluator SEQDO = new TransformEvaluator() { + @Override + public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { + PObjectValueTuple inputValues = context.getPObjectTuple(transform); + PObjectValueTuple outputValues = transform.getFn().process(inputValues); + context.setPObjectTuple(transform, outputValues); + } + }; + private static TransformEvaluator GBK = new TransformEvaluator() { @Override public void evaluate(GroupByKey transform, EvaluationContext context) { @@ -257,10 +272,12 @@ public static void registerEvaluator( registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); registerEvaluator(ParDo.Bound.class, PARDO); + registerEvaluator(SeqDo.BoundMulti.class, SEQDO); registerEvaluator(GroupByKey.class, GBK); registerEvaluator(Combine.GroupedValues.class, GROUPED); registerEvaluator(Flatten.class, FLATTEN); registerEvaluator(Create.class, CREATE); registerEvaluator(CreatePObject.class, CREATE_POBJ); + registerEvaluator(Convert.ToIterable.class, TO_ITER); } } diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 26c340b6bdab..83222be1bdd5 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -17,14 +17,20 @@ */ package com.cloudera.dataflow.spark; +import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.PipelineResult; -import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +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.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; 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.PObject; import org.junit.Test; public class WordCountTest { @@ -42,43 +48,30 @@ public void processElement(ProcessContext c) { } } - /** A DoFn that converts a Word and Count into a printable string. */ - static class FormatCountsFn extends DoFn, String> { + public static class CountWords extends PTransform, PCollection>> { @Override - public void processElement(ProcessContext c) { - c.output(c.element().getKey() + ": " + c.element().getValue()); - } - } - - public static class CountWords extends PTransform, PCollection> { - @Override - public PCollection apply(PCollection lines) { - + public PCollection> apply(PCollection lines) { // Convert lines of text into individual words. PCollection words = lines.apply( ParDo.of(new ExtractWordsFn())); - // Count the number of times each word occurs. - PCollection> wordCounts = - words.apply(Count.create()); - - // Format each word and count into a printable string. - PCollection results = wordCounts.apply( - ParDo.of(new FormatCountsFn())); - - return results; + return words.apply(Count.create()); } } @Test public void testRun() throws Exception { - SparkPipeline p = SparkPipeline.create("local"); + Pipeline p = Pipeline.create(new PipelineOptions()); + PCollection w1 = p.apply(Create.of("Here are some words to count")); + PCollection w2 = p.apply(Create.of("Here are some more words")); + PCollectionList list = PCollectionList.of(w1).and(w2); - p.apply(TextIO.Read.named("ReadLines").from("/tmp/test.txt")) - .apply(new CountWords()) - .apply(TextIO.Write.named("WriteCounts").to("/tmp/dfout.txt")); + PCollection union = list.apply(Flatten.create()); + PCollection> counts = union.apply(new CountWords()); + PObject unique = counts.apply(ApproximateUnique.>globally(16)); - PipelineResult result = p.run(); - System.out.println(result); + EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + System.out.println(res.get(counts)); + System.out.println(res.get(unique)); } } From 6ee38b2a0677fc8eb77b4cd0aca64cfec2ec7fd9 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 16:31:31 -0800 Subject: [PATCH 008/137] Support for ParDo.BoundMulti --- .../dataflow/spark/CoderSparkSerializer.java | 67 --------- .../cloudera/dataflow/spark/DoFnFunction.java | 2 +- .../dataflow/spark/EvaluationContext.java | 17 ++- .../dataflow/spark/MultiDoFnFunction.java | 127 ++++++++++++++++++ .../dataflow/spark/SparkPipelineRunner.java | 75 +++++++++-- .../dataflow/spark/WordCountTest.java | 1 - 6 files changed, 202 insertions(+), 87 deletions(-) delete mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.java deleted file mode 100644 index 3430d564f4d2..000000000000 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderSparkSerializer.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 com.cloudera.dataflow.spark; - -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.CoderRegistry; -import org.apache.spark.serializer.DeserializationStream; -import org.apache.spark.serializer.SerializationStream; -import org.apache.spark.serializer.Serializer; -import org.apache.spark.serializer.SerializerInstance; -import scala.reflect.ClassTag; - -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.ByteBuffer; - -public class CoderSparkSerializer extends Serializer implements Serializable { - - @Override - public SerializerInstance newInstance() { - return new CoderSerializerInstance(); - } - - private class CoderSerializerInstance extends SerializerInstance implements Serializable { - - @Override - public ByteBuffer serialize(T t, ClassTag ev) { - return null; - } - - @Override - public T deserialize(ByteBuffer bytes, ClassTag ev) { - return null; - } - - @Override - public T deserialize(ByteBuffer bytes, ClassLoader loader, ClassTag ev) { - return null; - } - - @Override - public SerializationStream serializeStream(OutputStream os) { - return null; - } - - @Override - public DeserializationStream deserializeStream(InputStream is) { - return null; - } - } -} 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 index 8bdeae4d682e..caed28b3cfa2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -32,7 +32,7 @@ import java.util.List; import java.util.Map; -public class DoFnFunction implements FlatMapFunction, O> { +class DoFnFunction implements FlatMapFunction, O> { private final DoFn fn; private final Map, BroadcastHelper> sideInputs; 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 index bd6ec952fde4..400b0748e9cd 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -20,7 +20,6 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.SeqDo; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.PObject; @@ -46,8 +45,8 @@ public class EvaluationContext implements EvaluationResult { final Set multireads = Sets.newHashSet(); final Map localPObjects = Maps.newHashMap(); - public EvaluationContext(String master, Pipeline pipeline) { - this.jsc = new JavaSparkContext(master, "dataflow"); + public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { + this.jsc = jsc; this.pipeline = pipeline; } @@ -56,12 +55,12 @@ JavaSparkContext getSparkContext() { } Pipeline getPipeline() { return pipeline; } - PInput getInput(PTransform transform) { - return pipeline.getInput(transform); + I getInput(PTransform transform) { + return (I) pipeline.getInput(transform); } - POutput getOutput(PTransform transform) { - return pipeline.getOutput(transform); + O getOutput(PTransform transform) { + return (O) pipeline.getOutput(transform); } void setOutputRDD(PTransform transform, JavaRDDLike rdd) { @@ -83,6 +82,10 @@ JavaRDDLike getRDD(PValue pvalue) { return rdd; } + void setRDD(PValue pvalue, JavaRDDLike rdd) { + rdds.put(pvalue, rdd); + } + JavaRDDLike getInputRDD(PTransform transform) { return getRDD((PValue) pipeline.getInput(transform)); } 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 new file mode 100644 index 000000000000..69cb93e3ed6b --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.streaming.KeyedState; +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.SerializableFunction; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.LinkedListMultimap; +import com.google.common.collect.Multimap; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import scala.Tuple2; + +import java.util.Iterator; +import java.util.Map; + +class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { + + private final DoFn fn; + private final TupleTag mainOutputTag; + private final Map, BroadcastHelper> sideInputs; + + public MultiDoFnFunction(DoFn fn, TupleTag mainOutputTag) { + this(fn, mainOutputTag, ImmutableMap., BroadcastHelper>of()); + } + + public MultiDoFnFunction(DoFn fn, + TupleTag mainOutputTag, + Map, BroadcastHelper> sideInputs) { + this.fn = fn; + this.mainOutputTag = mainOutputTag; + this.sideInputs = sideInputs; + } + + @Override + public Iterable, Object>> call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt(fn); + fn.startBatch(ctxt); + while (iter.hasNext()) { + ctxt.element = iter.next(); + fn.processElement(ctxt); + } + fn.finishBatch(ctxt); + return Iterables.transform(ctxt.outputs.entries(), + new Function, Object>, Tuple2, Object>>() { + public Tuple2, Object> apply(Map.Entry, Object> input) { + return new Tuple2, Object>(input.getKey(), input.getValue()); + } + }); + } + + private class ProcCtxt extends DoFn.ProcessContext { + + private Multimap, Object> outputs = LinkedListMultimap.create(); + private I element; + + public ProcCtxt(DoFn fn) { + fn.super(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return null; + } + + @Override + public synchronized void output(O o) { + outputs.put(mainOutputTag, o); + } + + @Override + public synchronized void sideOutput(TupleTag tag, T t) { + outputs.put(tag, t); + } + + @Override + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return null; + } + + @Override + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return null; + } + + @Override + public T sideInput(TupleTag tag) { + BroadcastHelper bh = (BroadcastHelper) sideInputs.get(tag); + return bh == null ? null : bh.getValue(); + } + + @Override + public I element() { + return element; + } + + @Override + public KeyedState keyedState() { + throw new UnsupportedOperationException(); + } + } +} 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 index 5d7a835247d3..74a712616c5f 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -36,6 +37,7 @@ 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.PObject; import com.google.cloud.dataflow.sdk.values.PObjectTuple; import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; @@ -45,8 +47,12 @@ 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.PairFunction; +import scala.Function1; +import scala.Option; +import scala.PartialFunction; import scala.Tuple2; import java.util.Map; @@ -69,12 +75,17 @@ public SparkPipelineRunner(String master) { @Override public EvaluationResult run(Pipeline pipeline) { - EvaluationContext ctxt = new EvaluationContext(this.master, pipeline); + JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); return ctxt; } - private class Evaluator implements Pipeline.PipelineVisitor { + private JavaSparkContext getContextFromOptions(PipelineOptions options) { + return new JavaSparkContext(master, options.getJobNameOrDefault()); + } + + private static class Evaluator implements Pipeline.PipelineVisitor { private final EvaluationContext ctxt; @@ -196,6 +207,56 @@ public void evaluate(ParDo.Bound transform, EvaluationContext context) { } }; + private static TransformEvaluator MULTIDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); + PObjectTuple pot = transform.getSideInputs(); + MultiDoFnFunction multifn; + if (pot == null || pot.getAll().isEmpty()) { + multifn = new MultiDoFnFunction(transform.getFn(), transform.getMainOutputTag()); + } else { + Map, BroadcastHelper> sideInputs = Maps.newHashMap(); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); + } + multifn = new MultiDoFnFunction(transform.getFn(), transform.getMainOutputTag(), sideInputs); + } + + JavaPairRDD all = last.mapPartitionsToPair(multifn); + all.cache(); + + PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); + for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { + TupleTagFilter filter = new TupleTagFilter(e.getKey()); + JavaPairRDD filtered = all.filter(filter); + context.setRDD(e.getValue(), filtered.values()); + } + } + }; + + private static class TupleTagFilter implements Function, Boolean> { + private TupleTag tag; + + public TupleTagFilter(TupleTag tag) { + this.tag = tag; + } + + @Override + public Boolean call(Tuple2 input) throws Exception { + return tag.equals(input._1()); + } + } + + private static TransformEvaluator SEQDO = new TransformEvaluator() { + @Override + public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { + PObjectValueTuple inputValues = context.getPObjectTuple(transform); + PObjectValueTuple outputValues = transform.getFn().process(inputValues); + context.setPObjectTuple(transform, outputValues); + } + }; + private static JavaPairRDD toPair(JavaRDDLike rdd) { return rdd.mapToPair(new PairFunction() { @Override @@ -216,15 +277,6 @@ public Object call(Object o) throws Exception { }); } - private static TransformEvaluator SEQDO = new TransformEvaluator() { - @Override - public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { - PObjectValueTuple inputValues = context.getPObjectTuple(transform); - PObjectValueTuple outputValues = transform.getFn().process(inputValues); - context.setPObjectTuple(transform, outputValues); - } - }; - private static TransformEvaluator GBK = new TransformEvaluator() { @Override public void evaluate(GroupByKey transform, EvaluationContext context) { @@ -272,6 +324,7 @@ public static void registerEvaluator( registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); registerEvaluator(ParDo.Bound.class, PARDO); + registerEvaluator(ParDo.BoundMulti.class, MULTIDO); registerEvaluator(SeqDo.BoundMulti.class, SEQDO); registerEvaluator(GroupByKey.class, GBK); registerEvaluator(Combine.GroupedValues.class, GROUPED); diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 83222be1bdd5..0e7358e7d48c 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -18,7 +18,6 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.runners.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.ApproximateUnique; import com.google.cloud.dataflow.sdk.transforms.Count; From 64c6d8d723023aafc2ee5754d41e00eca1524ddf Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 16:54:14 -0800 Subject: [PATCH 009/137] Fix bug in deserializing side inputs --- .../dataflow/spark/BroadcastHelper.java | 3 +-- .../dataflow/spark/EvaluationContext.java | 10 ++++---- .../dataflow/spark/WordCountTest.java | 25 ++++++++++++++++--- 3 files changed, 28 insertions(+), 10 deletions(-) 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 index 670bbc8de5eb..48422e469283 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -43,10 +43,9 @@ public synchronized T getValue() { private T deserialize() { try { - coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); + return coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); } catch (IOException e) { throw new RuntimeException("Error deserializing broadcast variable", e); } - return null; } } 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 index 400b0748e9cd..ede65a712f66 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -43,7 +43,7 @@ public class EvaluationContext implements EvaluationResult { final Pipeline pipeline; final Map rdds = Maps.newHashMap(); final Set multireads = Sets.newHashSet(); - final Map localPObjects = Maps.newHashMap(); + final Map pobjects = Maps.newHashMap(); public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; @@ -68,7 +68,7 @@ void setOutputRDD(PTransform transform, JavaRDDLike rdd) { } void setPObjectValue(PObject pobj, Object value) { - localPObjects.put(pobj, value); + pobjects.put(pobj, value); } JavaRDDLike getRDD(PValue pvalue) { @@ -98,13 +98,13 @@ BroadcastHelper getBroadcastHelper(PObject value) { @Override public T get(PObject value) { - if (localPObjects.containsKey(value)) { - return (T) localPObjects.get(value); + if (pobjects.containsKey(value)) { + return (T) pobjects.get(value); } else if (rdds.containsKey(value)) { JavaRDDLike rdd = rdds.get(value); //TODO: probably some work to do here T res = (T) Iterables.getOnlyElement(rdd.collect()); - localPObjects.put(value, res); + pobjects.put(value, res); return res; } throw new IllegalStateException("Cannot resolve un-known PObject: " + value); diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 0e7358e7d48c..3edbafb1de9d 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -22,6 +22,7 @@ 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.CreatePObject; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.Flatten; import com.google.cloud.dataflow.sdk.transforms.PTransform; @@ -30,15 +31,23 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.cloud.dataflow.sdk.values.PObject; +import com.google.cloud.dataflow.sdk.values.PObjectTuple; +import com.google.cloud.dataflow.sdk.values.TupleTag; import org.junit.Test; public class WordCountTest { /** A DoFn that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { + + private TupleTag regex; + + public ExtractWordsFn(TupleTag regex) { + this.regex = regex; + } @Override public void processElement(ProcessContext c) { - String[] words = c.element().split("[^a-zA-Z']+"); + String[] words = c.element().split(c.sideInput(regex)); for (String word : words) { if (!word.isEmpty()) { c.output(word); @@ -48,11 +57,20 @@ public void processElement(ProcessContext c) { } public static class CountWords extends PTransform, PCollection>> { + + private final PObject regex; + + public CountWords(PObject regex) { + this.regex = regex; + } + @Override public PCollection> apply(PCollection lines) { // Convert lines of text into individual words. + TupleTag tag = new TupleTag(); PCollection words = lines.apply( - ParDo.of(new ExtractWordsFn())); + ParDo.of(new ExtractWordsFn(tag)) + .withSideInputs(PObjectTuple.of(tag, regex))); // Count the number of times each word occurs. return words.apply(Count.create()); } @@ -61,12 +79,13 @@ public PCollection> apply(PCollection lines) { @Test public void testRun() throws Exception { Pipeline p = Pipeline.create(new PipelineOptions()); + PObject regex = p.apply(CreatePObject.of("[^a-zA-Z']+")); PCollection w1 = p.apply(Create.of("Here are some words to count")); PCollection w2 = p.apply(Create.of("Here are some more words")); PCollectionList list = PCollectionList.of(w1).and(w2); PCollection union = list.apply(Flatten.create()); - PCollection> counts = union.apply(new CountWords()); + PCollection> counts = union.apply(new CountWords(regex)); PObject unique = counts.apply(ApproximateUnique.>globally(16)); EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); From 40adbeceb32b5d8a2a01bb5156c4e97ae9cfc58c Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 19:13:24 -0800 Subject: [PATCH 010/137] Add SparkRuntimeContext for handling shared runtime objects --- .../cloudera/dataflow/spark/DoFnFunction.java | 13 ++-- .../dataflow/spark/EvaluationContext.java | 13 ++-- .../dataflow/spark/MultiDoFnFunction.java | 17 ++--- .../dataflow/spark/SparkPipelineRunner.java | 56 +++++++------- .../dataflow/spark/SparkRuntimeContext.java | 73 +++++++++++++++++++ .../dataflow/spark/WordCountTest.java | 49 ++++++++----- 6 files changed, 153 insertions(+), 68 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.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 index caed28b3cfa2..636f561f1bc5 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -35,14 +35,15 @@ class DoFnFunction implements FlatMapFunction, O> { private final DoFn fn; + private final SparkRuntimeContext runtimeContext; private final Map, BroadcastHelper> sideInputs; - public DoFnFunction(DoFn fn) { - this(fn, ImmutableMap., BroadcastHelper>of()); - } - - public DoFnFunction(DoFn fn, Map, BroadcastHelper> sideInputs) { + public DoFnFunction( + DoFn fn, + SparkRuntimeContext runtime, + Map, BroadcastHelper> sideInputs) { this.fn = fn; + this.runtimeContext = runtime; this.sideInputs = sideInputs; } @@ -69,7 +70,7 @@ public ProcCtxt(DoFn fn) { @Override public PipelineOptions getPipelineOptions() { - return null; + return runtimeContext.getPipelineOptions(); } @Override 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 index ede65a712f66..57f1622d3ada 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -39,21 +39,24 @@ import java.util.Set; public class EvaluationContext implements EvaluationResult { - final JavaSparkContext jsc; - final Pipeline pipeline; - final Map rdds = Maps.newHashMap(); - final Set multireads = Sets.newHashSet(); - final Map pobjects = Maps.newHashMap(); + private final JavaSparkContext jsc; + private final Pipeline pipeline; + private final SparkRuntimeContext runtime; + private final Map rdds = Maps.newHashMap(); + private final Set multireads = Sets.newHashSet(); + private final Map pobjects = Maps.newHashMap(); public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; this.pipeline = pipeline; + this.runtime = new SparkRuntimeContext(jsc, pipeline); } JavaSparkContext getSparkContext() { return jsc; } Pipeline getPipeline() { return pipeline; } + SparkRuntimeContext getRuntimeContext() { return runtime; } I getInput(PTransform transform) { return (I) pipeline.getInput(transform); 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 index 69cb93e3ed6b..b7e647496b48 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -25,7 +25,6 @@ import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; @@ -38,17 +37,17 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { private final DoFn fn; + private final SparkRuntimeContext runtimeContext; private final TupleTag mainOutputTag; private final Map, BroadcastHelper> sideInputs; - public MultiDoFnFunction(DoFn fn, TupleTag mainOutputTag) { - this(fn, mainOutputTag, ImmutableMap., BroadcastHelper>of()); - } - - public MultiDoFnFunction(DoFn fn, + public MultiDoFnFunction( + DoFn fn, + SparkRuntimeContext runtimeContext, TupleTag mainOutputTag, Map, BroadcastHelper> sideInputs) { this.fn = fn; + this.runtimeContext = runtimeContext; this.mainOutputTag = mainOutputTag; this.sideInputs = sideInputs; } @@ -81,7 +80,7 @@ public ProcCtxt(DoFn fn) { @Override public PipelineOptions getPipelineOptions() { - return null; + return runtimeContext.getPipelineOptions(); } @Override @@ -98,14 +97,14 @@ public synchronized void sideOutput(TupleTag tag, T t) { public Aggregator createAggregator( String named, Combine.CombineFn combineFn) { - return null; + return runtimeContext.createAggregator(named, combineFn); } @Override public Aggregator createAggregator( String named, SerializableFunction, AO> sfunc) { - return null; + return runtimeContext.createAggregator(named, sfunc); } @Override 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 index 74a712616c5f..7394a3d452fe 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -44,6 +44,7 @@ import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; @@ -188,43 +189,42 @@ public void evaluate(Convert.ToIterable transform, EvaluationContext context) { } }; + private static Map, BroadcastHelper> getSideInputs( + PObjectTuple pot, + EvaluationContext context) { + if (pot == null || pot.getAll().isEmpty()) { + return ImmutableMap.of(); + } else { + Map, BroadcastHelper>sideInputs = Maps.newHashMap(); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); + } + return sideInputs; + } + } + private static TransformEvaluator PARDO = new TransformEvaluator() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - PObjectTuple pot = transform.getSideInputs(); - DoFnFunction dofn; - if (pot == null || pot.getAll().isEmpty()) { - dofn = new DoFnFunction(transform.getFn()); - } else { - Map, BroadcastHelper> sideInputs = Maps.newHashMap(); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); - } - dofn = new DoFnFunction(transform.getFn(), sideInputs); - } - context.setOutputRDD(transform, last.mapPartitions(dofn)); + DoFnFunction dofn = new DoFnFunction(transform.getFn(), + context.getRuntimeContext(), + getSideInputs(transform.getSideInputs(), context)); + context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); } }; private static TransformEvaluator MULTIDO = new TransformEvaluator() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - PObjectTuple pot = transform.getSideInputs(); - MultiDoFnFunction multifn; - if (pot == null || pot.getAll().isEmpty()) { - multifn = new MultiDoFnFunction(transform.getFn(), transform.getMainOutputTag()); - } else { - Map, BroadcastHelper> sideInputs = Maps.newHashMap(); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); - } - multifn = new MultiDoFnFunction(transform.getFn(), transform.getMainOutputTag(), sideInputs); - } - - JavaPairRDD all = last.mapPartitionsToPair(multifn); - all.cache(); + MultiDoFnFunction multifn = new MultiDoFnFunction( + transform.getFn(), + context.getRuntimeContext(), + transform.getMainOutputTag(), + getSideInputs(transform.getSideInputs(), context)); + + JavaPairRDD all = context.getInputRDD(transform) + .mapPartitionsToPair(multifn) + .cache(); PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { 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 new file mode 100644 index 000000000000..5df69b0b9de9 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.SerializationConfig; + +import java.io.IOException; +import java.io.Serializable; + +class SparkRuntimeContext implements Serializable { + + private transient PipelineOptions pipelineOptions; + private Broadcast jsonOptions; + + public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { + this.jsonOptions = jsc.broadcast(optionsToJson(pipeline.getOptions())); + } + + private static String optionsToJson(PipelineOptions options) { + try { + return createMapper().writeValueAsString(options); + } catch (IOException e) { + throw new RuntimeException("Could not write PipelineOptions as JSON", e); + } + } + + public synchronized PipelineOptions getPipelineOptions() { + return null; + } + + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return null; + } + + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return null; + } + + private static ObjectMapper createMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(SerializationConfig.Feature.FAIL_ON_EMPTY_BEANS, false); + mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return mapper; + } +} diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 3edbafb1de9d..90e4923187ce 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -39,40 +39,49 @@ public class WordCountTest { /** A DoFn that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { - - private TupleTag regex; - - public ExtractWordsFn(TupleTag regex) { - this.regex = regex; - } @Override public void processElement(ProcessContext c) { String[] words = c.element().split(c.sideInput(regex)); for (String word : words) { if (!word.isEmpty()) { + if (Character.isLowerCase(word.charAt(0))) c.output(word); + } else { + c.sideOutput(upper, word); } } } } + static TupleTag regex = new TupleTag<>(); + static TupleTag upper = new TupleTag<>(); + static TupleTag lower = new TupleTag<>(); + static TupleTag> lowerCnts = new TupleTag<>(); + static TupleTag> upperCnts = new TupleTag<>(); + public static class CountWords extends PTransform, PCollection>> { - private final PObject regex; + private final PObject regexObj; - public CountWords(PObject regex) { - this.regex = regex; + public CountWords(PObject regexObj) { + this.regexObj = regexObj; } @Override public PCollection> apply(PCollection lines) { // Convert lines of text into individual words. - TupleTag tag = new TupleTag(); - PCollection words = lines.apply( - ParDo.of(new ExtractWordsFn(tag)) - .withSideInputs(PObjectTuple.of(tag, regex))); - // Count the number of times each word occurs. - return words.apply(Count.create()); + return lines + .apply(ParDo.of(new ExtractWordsFn()) + .withSideInputs(PObjectTuple.of(regex, regexObj))) + .apply(Count.create()); + //.withOutputTags(lower, TupleTagList.of(upper))); + /* + PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.create()); + PCollection> upperCounts = lowerUpper.get(upper).apply(Count.create()); + return PCollectionTuple + .of(lowerCnts, lowerCounts) + .and(upperCnts, upperCounts); + */ } } @@ -80,16 +89,16 @@ public PCollection> apply(PCollection lines) { public void testRun() throws Exception { Pipeline p = Pipeline.create(new PipelineOptions()); PObject regex = p.apply(CreatePObject.of("[^a-zA-Z']+")); - PCollection w1 = p.apply(Create.of("Here are some words to count")); - PCollection w2 = p.apply(Create.of("Here are some more words")); + PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); + PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); PCollectionList list = PCollectionList.of(w1).and(w2); PCollection union = list.apply(Flatten.create()); - PCollection> counts = union.apply(new CountWords(regex)); - PObject unique = counts.apply(ApproximateUnique.>globally(16)); + PCollection> lowerCounts = union.apply(new CountWords(regex)); + PObject unique = lowerCounts.apply(ApproximateUnique.>globally(16)); EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); - System.out.println(res.get(counts)); + System.out.println(res.get(lowerCounts)); System.out.println(res.get(unique)); } } From 565509dc50a0f216665e6900a65842509b63e4d5 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 20:12:49 -0800 Subject: [PATCH 011/137] First cut at aggregators --- .../java/com/cloudera/dataflow/spark/Agg.java | 130 ++++++++++++++++++ .../dataflow/spark/AggAccumParam.java | 37 +++++ .../dataflow/spark/SparkRuntimeContext.java | 42 +++++- 3 files changed, 205 insertions(+), 4 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java new file mode 100644 index 000000000000..827591ef1352 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java @@ -0,0 +1,130 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.common.collect.ImmutableList; + +import java.io.Serializable; +import java.util.Map; +import java.util.TreeMap; + +public class Agg implements Serializable { + + public interface State extends Serializable { + void update(VI vi); + State merge(State other); + VA current(); + VO render(); + } + + private final Map states = new TreeMap<>(); + + public Agg() { + } + + public Agg(String named, State state) { + this.states.put(named, state); + } + + public Agg merge(Agg other) { + for (Map.Entry e : other.states.entrySet()) { + State cur = states.get(e.getKey()); + if (cur == null) { + states.put(e.getKey(), e.getValue()); + } else { + states.put(e.getKey(), cur.merge(e.getValue())); + } + } + return this; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (Map.Entry e : states.entrySet()) { + sb.append(e.getKey()).append(": ").append(e.getValue().render()); + } + return sb.toString(); + } + + public static class CombineState implements State { + + private Combine.CombineFn combineFn; + private VA state; + + public CombineState(Combine.CombineFn combineFn) { + this.combineFn = combineFn; + this.state = combineFn.createAccumulator(); + } + + @Override + public void update(VI vi) { + combineFn.addInput(state, vi); + } + + @Override + public State merge(State other) { + this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); + return this; + } + + @Override + public VA current() { + return state; + } + + @Override + public VO render() { + return combineFn.extractOutput(state); + } + } + + public static class SerState implements State { + + private SerializableFunction, VO> sfunc; + private VO state; + + public SerState(SerializableFunction, VO> sfunc) { + this.sfunc = sfunc; + } + + @Override + public void update(VI vi) { + this.state = sfunc.apply(ImmutableList.of(vi, (VI) state)); + } + + @Override + public State merge(State other) { + this.state = sfunc.apply(ImmutableList.of((VI) state, (VI) other.current())); + return this; + } + + @Override + public VO current() { + return state; + } + + @Override + public VO render() { + return state; + } + } + +} diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java new file mode 100644 index 000000000000..221ad03e95e1 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark; + +import org.apache.spark.AccumulatorParam; + +public class AggAccumParam implements AccumulatorParam { + @Override + public Agg addAccumulator(Agg current, Agg added) { + return current.merge(added); + } + + @Override + public Agg addInPlace(Agg current, Agg added) { + return addAccumulator(current, added); + } + + @Override + public Agg zero(Agg initialValue) { + return new Agg(); + } +} 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 index 5df69b0b9de9..c21083554afb 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -22,6 +22,8 @@ import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.common.collect.ImmutableMap; +import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.codehaus.jackson.map.DeserializationConfig; @@ -30,14 +32,19 @@ import java.io.IOException; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; class SparkRuntimeContext implements Serializable { private transient PipelineOptions pipelineOptions; private Broadcast jsonOptions; + private Accumulator accum; + private Map aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsonOptions = jsc.broadcast(optionsToJson(pipeline.getOptions())); + this.accum = jsc.accumulator(new Agg(), new AggAccumParam()); } private static String optionsToJson(PipelineOptions options) { @@ -52,16 +59,30 @@ public synchronized PipelineOptions getPipelineOptions() { return null; } - public Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, SerializableFunction, AO> sfunc) { - return null; + Aggregator aggregator = aggregators.get(named); + if (aggregator == null) { + Agg.SerState state = new Agg.SerState<>(sfunc); + accum.add(new Agg(named, state)); + aggregator = new SparkAggregator(state); + aggregators.put(named, aggregator); + } + return aggregator; } - public Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, Combine.CombineFn combineFn) { - return null; + Aggregator aggregator = aggregators.get(named); + if (aggregator == null) { + Agg.CombineState state = new Agg.CombineState<>(combineFn); + accum.add(new Agg(named, state)); + aggregator = new SparkAggregator(state); + aggregators.put(named, aggregator); + } + return aggregator; } private static ObjectMapper createMapper() { @@ -70,4 +91,17 @@ private static ObjectMapper createMapper() { mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); return mapper; } + + private static class SparkAggregator implements Aggregator { + private final Agg.State state; + + public SparkAggregator(Agg.State state) { + this.state = state; + } + + @Override + public void addValue(VI vi) { + state.update(vi); + } + } } From bb219d48a89f4f61d090b7cf47566dcee85adcf9 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 2 Dec 2014 21:03:35 -0800 Subject: [PATCH 012/137] First minimally working aggregators --- .../java/com/cloudera/dataflow/spark/Agg.java | 5 +++++ .../cloudera/dataflow/spark/DoFnFunction.java | 4 ++-- .../dataflow/spark/EvaluationContext.java | 5 +++++ .../dataflow/spark/EvaluationResult.java | 1 + .../dataflow/spark/SparkRuntimeContext.java | 4 ++++ .../dataflow/spark/WordCountTest.java | 21 ++++++++++++++++++- 6 files changed, 37 insertions(+), 3 deletions(-) diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java index 827591ef1352..35861a003172 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java @@ -43,6 +43,10 @@ public Agg(String named, State state) { this.states.put(named, state); } + public T getValue(String named, Class typeClass) { + return typeClass.cast(states.get(named).render()); + } + public Agg merge(Agg other) { for (Map.Entry e : other.states.entrySet()) { State cur = states.get(e.getKey()); @@ -103,6 +107,7 @@ public static class SerState implements State { public SerState(SerializableFunction, VO> sfunc) { this.sfunc = sfunc; + this.state = sfunc.apply(ImmutableList.of()); } @Override 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 index 636f561f1bc5..377343d0a323 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -87,14 +87,14 @@ public void sideOutput(TupleTag tupleTag, T t) { public Aggregator createAggregator( String named, Combine.CombineFn combineFn) { - return null; + return runtimeContext.createAggregator(named, combineFn); } @Override public Aggregator createAggregator( String named, SerializableFunction, AO> sfunc) { - return null; + return runtimeContext.createAggregator(named, sfunc); } @Override 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 index 57f1622d3ada..fd065b2db489 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -113,6 +113,11 @@ public T get(PObject value) { throw new IllegalStateException("Cannot resolve un-known PObject: " + value); } + @Override + public T getAggregatorValue(String named, Class resultType) { + return runtime.getAggregatorValue(named, resultType); + } + @Override public Iterable get(PCollection pcollection) { return getRDD(pcollection).collect(); 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 index 94a50df0100b..74fe20549fc1 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -24,4 +24,5 @@ public interface EvaluationResult extends PipelineResult { Iterable get(PCollection pcollection); T get(PObject pobject); + T getAggregatorValue(String named, Class resultType); } 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 index c21083554afb..fdf05af09b8c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -47,6 +47,10 @@ public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new Agg(), new AggAccumParam()); } + public T getAggregatorValue(String named, Class typeClass) { + return accum.value().getValue(named, typeClass); + } + private static String optionsToJson(PipelineOptions options) { try { return createMapper().writeValueAsString(options); diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 90e4923187ce..f7a90a60bcbd 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -19,14 +19,17 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +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.CreatePObject; 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.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; @@ -39,11 +42,25 @@ public class WordCountTest { /** A DoFn that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { + + Aggregator totalWords; + Aggregator maxWordLength; + + @Override + public void startBatch(Context ctxt) { + this.totalWords = ctxt.createAggregator("totalWords", + new Sum.SumIntegerFn()); + this.maxWordLength = ctxt.createAggregator("maxWordLength", + new Max.MaxIntegerFn()); + } + @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 { @@ -72,7 +89,7 @@ public PCollection> apply(PCollection lines) { // Convert lines of text into individual words. return lines .apply(ParDo.of(new ExtractWordsFn()) - .withSideInputs(PObjectTuple.of(regex, regexObj))) + .withSideInputs(PObjectTuple.of(regex, regexObj))) .apply(Count.create()); //.withOutputTags(lower, TupleTagList.of(upper))); /* @@ -100,5 +117,7 @@ public void testRun() throws Exception { EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); System.out.println(res.get(lowerCounts)); System.out.println(res.get(unique)); + System.out.println(res.getAggregatorValue("totalWords", Integer.class)); + System.out.println(res.getAggregatorValue("maxWordLength", Integer.class)); } } From 9152769c79c00aa8c9332c23ffcdd42a9876f402 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 8 Dec 2014 22:40:34 -0800 Subject: [PATCH 013/137] Updates for 141206 SDK release --- runners/spark/pom.xml | 37 +++------- .../cloudera/dataflow/spark/DoFnFunction.java | 41 +++++++---- .../dataflow/spark/EvaluationContext.java | 8 +- .../dataflow/spark/MultiDoFnFunction.java | 38 +++++++--- .../dataflow/spark/SparkPipelineRunner.java | 73 +++++++++++++++---- .../dataflow/spark/SparkRuntimeContext.java | 28 +------ .../dataflow/spark/WordCountTest.java | 71 ++++++++++-------- 7 files changed, 170 insertions(+), 126 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 1ac764b5873e..fc07069846d1 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -18,30 +18,16 @@ 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"> 4.0.0 - Cloud Dataflow Examples - com.google.cloud.dataflow - examples - 1 + Dataflow on Spark + com.cloudera.dataflow.spark + dataflow-spark + 0.1.0 jar UTF-8 - - - dataflow-snapshot - file://${project.basedir}/jars - - true - - - true - always - - - - @@ -89,11 +75,15 @@ spark-core_2.10 1.1.0 - + + com.google.guava + guava + 18.0 + com.google.cloud.dataflow dataflow-sdk - 0.0.1 + 1.0.141206 com.google.apis @@ -131,13 +121,6 @@ 2.4.2 - - - commons-cli - commons-cli - 1.2 - - org.hamcrest 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 index 377343d0a323..54a2a78fe753 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -17,16 +17,18 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; -import com.google.cloud.dataflow.sdk.streaming.KeyedState; +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.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +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.spark.api.java.function.FlatMapFunction; +import org.joda.time.Instant; +import java.util.Collection; import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -50,12 +52,12 @@ public DoFnFunction( @Override public Iterable call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(fn); - fn.startBatch(ctxt); + fn.startBundle(ctxt); while (iter.hasNext()) { ctxt.element = iter.next(); fn.processElement(ctxt); } - fn.finishBatch(ctxt); + fn.finishBundle(ctxt); return ctxt.outputs; } @@ -73,6 +75,11 @@ public PipelineOptions getPipelineOptions() { return runtimeContext.getPipelineOptions(); } + @Override + public T sideInput(PCollectionView view) { + return (T) sideInputs.get(view.getTagInternal()).getValue(); + } + @Override public synchronized void output(O o) { outputs.add(o); @@ -80,7 +87,7 @@ public synchronized void output(O o) { @Override public void sideOutput(TupleTag tupleTag, T t) { - // A no-op if we don't know about it ahead of time + // A no-op in this context; maybe add some logging } @Override @@ -97,20 +104,28 @@ public Aggregator createAggregator( return runtimeContext.createAggregator(named, sfunc); } - @Override - public T sideInput(TupleTag tag) { - BroadcastHelper bh = (BroadcastHelper) sideInputs.get(tag); - return bh == null ? null : bh.getValue(); - } - @Override public I element() { return element; } @Override - public KeyedState keyedState() { + public DoFn.KeyedState keyedState() { throw new UnsupportedOperationException(); } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + } + + @Override + public Instant timestamp() { + return null; + } + + @Override + public Collection windows() { + return null; + } } } 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 index fd065b2db489..f30aa809547e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -70,9 +70,9 @@ void setOutputRDD(PTransform transform, JavaRDDLike rdd) { rdds.put((PValue) getOutput(transform), rdd); } - void setPObjectValue(PObject pobj, Object value) { - pobjects.put(pobj, value); - } + void setPObjectValue(PObject pobject, Object value) { + pobjects.put(pobject, value); + } JavaRDDLike getRDD(PValue pvalue) { JavaRDDLike rdd = rdds.get(pvalue); @@ -135,7 +135,7 @@ PObjectValueTuple getPObjectTuple(PTransform transform) { void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - setPObjectValue(e.getValue(), outputValues.get(e.getKey())); + pobjects.put(e.getValue(), outputValues.get(e.getKey())); } } } 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 index b7e647496b48..54383a3f8a04 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -17,20 +17,23 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; -import com.google.cloud.dataflow.sdk.streaming.KeyedState; +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.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; import com.google.common.collect.Iterables; 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; +import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -55,12 +58,12 @@ public MultiDoFnFunction( @Override public Iterable, Object>> call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(fn); - fn.startBatch(ctxt); + fn.startBundle(ctxt); while (iter.hasNext()) { ctxt.element = iter.next(); fn.processElement(ctxt); } - fn.finishBatch(ctxt); + fn.finishBundle(ctxt); return Iterables.transform(ctxt.outputs.entries(), new Function, Object>, Tuple2, Object>>() { public Tuple2, Object> apply(Map.Entry, Object> input) { @@ -83,6 +86,11 @@ public PipelineOptions getPipelineOptions() { return runtimeContext.getPipelineOptions(); } + @Override + public T sideInput(PCollectionView view) { + return (T) sideInputs.get(view.getTagInternal()).getValue(); + } + @Override public synchronized void output(O o) { outputs.put(mainOutputTag, o); @@ -107,20 +115,28 @@ public Aggregator createAggregator( return runtimeContext.createAggregator(named, sfunc); } - @Override - public T sideInput(TupleTag tag) { - BroadcastHelper bh = (BroadcastHelper) sideInputs.get(tag); - return bh == null ? null : bh.getValue(); - } - @Override public I element() { return element; } @Override - public KeyedState keyedState() { + public DoFn.KeyedState keyedState() { throw new UnsupportedOperationException(); } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + } + + @Override + public Instant timestamp() { + return null; + } + + @Override + public Collection windows() { + return null; + } } } 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 index 7394a3d452fe..e995c2d302e0 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -22,7 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -34,28 +34,28 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SeqDo; +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.PObject; -import com.google.cloud.dataflow.sdk.values.PObjectTuple; import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; 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.PairFunction; -import scala.Function1; -import scala.Option; -import scala.PartialFunction; import scala.Tuple2; +import java.lang.reflect.Field; import java.util.Map; import java.util.logging.Logger; @@ -83,7 +83,7 @@ public EvaluationResult run(Pipeline pipeline) { } private JavaSparkContext getContextFromOptions(PipelineOptions options) { - return new JavaSparkContext(master, options.getJobNameOrDefault()); + return new JavaSparkContext(master, options.getJobName()); } private static class Evaluator implements Pipeline.PipelineVisitor { @@ -119,6 +119,30 @@ public void visitValue(PValue pvalue, TransformTreeNode node) { } } + private static class FieldGetter { + private Map fields; + + public FieldGetter(Class clazz) { + this.fields = Maps.newHashMap(); + for (Field f : clazz.getDeclaredFields()) { + try { + f.setAccessible(true); + this.fields.put(f.getName(), f); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + public T get(String fieldname, Object value) { + try { + return (T) fields.get(fieldname).get(value); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + private static TransformEvaluator READ_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { @@ -189,15 +213,31 @@ public void evaluate(Convert.ToIterable transform, EvaluationContext context) { } }; + private static TransformEvaluator TO_ITER_WIN = + new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, Iterables.transform(context.get(in), + new com.google.common.base.Function() { + @Override + public WindowedValue apply(Object o) { + return WindowedValue.valueInGlobalWindow(o); + } + })); + } + }; + private static Map, BroadcastHelper> getSideInputs( - PObjectTuple pot, + Iterable> views, EvaluationContext context) { - if (pot == null || pot.getAll().isEmpty()) { + if (views == null) { return ImmutableMap.of(); } else { Map, BroadcastHelper>sideInputs = Maps.newHashMap(); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - sideInputs.put(e.getKey(), context.getBroadcastHelper(e.getValue())); + for (PCollectionView view : views) { + sideInputs.put(view.getTagInternal(), context.getBroadcastHelper(view.getPObjectInternal())); } return sideInputs; } @@ -213,13 +253,14 @@ public void evaluate(ParDo.Bound transform, EvaluationContext context) { } }; + private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); private static TransformEvaluator MULTIDO = new TransformEvaluator() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { MultiDoFnFunction multifn = new MultiDoFnFunction( transform.getFn(), context.getRuntimeContext(), - transform.getMainOutputTag(), + (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), getSideInputs(transform.getSideInputs(), context)); JavaPairRDD all = context.getInputRDD(transform) @@ -277,17 +318,18 @@ public Object call(Object o) throws Exception { }); } - private static TransformEvaluator GBK = new TransformEvaluator() { + private static TransformEvaluator GBK = new TransformEvaluator() { @Override - public void evaluate(GroupByKey transform, EvaluationContext context) { + public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); } }; + private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); private static TransformEvaluator GROUPED = new TransformEvaluator() { @Override public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - final Combine.KeyedCombineFn keyed = transform.getFn(); + final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { @Override public Object call(Object input) throws Exception { @@ -326,11 +368,12 @@ public static void registerEvaluator( registerEvaluator(ParDo.Bound.class, PARDO); registerEvaluator(ParDo.BoundMulti.class, MULTIDO); registerEvaluator(SeqDo.BoundMulti.class, SEQDO); - registerEvaluator(GroupByKey.class, GBK); + registerEvaluator(GroupByKey.GroupByKeyOnly.class, GBK); registerEvaluator(Combine.GroupedValues.class, GROUPED); registerEvaluator(Flatten.class, FLATTEN); registerEvaluator(Create.class, CREATE); registerEvaluator(CreatePObject.class, CREATE_POBJ); registerEvaluator(Convert.ToIterable.class, TO_ITER); + registerEvaluator(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); } } 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 index fdf05af09b8c..fb405f08795d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -18,32 +18,23 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +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.SerializableFunction; -import com.google.common.collect.ImmutableMap; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.broadcast.Broadcast; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; -import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.Map; class SparkRuntimeContext implements Serializable { - private transient PipelineOptions pipelineOptions; - private Broadcast jsonOptions; private Accumulator accum; private Map aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.jsonOptions = jsc.broadcast(optionsToJson(pipeline.getOptions())); this.accum = jsc.accumulator(new Agg(), new AggAccumParam()); } @@ -51,16 +42,8 @@ public T getAggregatorValue(String named, Class typeClass) { return accum.value().getValue(named, typeClass); } - private static String optionsToJson(PipelineOptions options) { - try { - return createMapper().writeValueAsString(options); - } catch (IOException e) { - throw new RuntimeException("Could not write PipelineOptions as JSON", e); - } - } - public synchronized PipelineOptions getPipelineOptions() { - return null; + return null; // TODO } public synchronized Aggregator createAggregator( @@ -89,13 +72,6 @@ public synchronized Aggregator createAggregator( return aggregator; } - private static ObjectMapper createMapper() { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(SerializationConfig.Feature.FAIL_ON_EMPTY_BEANS, false); - mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return mapper; - } - private static class SparkAggregator implements Aggregator { private final Agg.State state; diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index f7a90a60bcbd..a04b41ea34fe 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -18,12 +18,12 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.runners.PipelineOptions; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; 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.CreatePObject; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.Flatten; import com.google.cloud.dataflow.sdk.transforms.Max; @@ -33,11 +33,15 @@ 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.PObject; -import com.google.cloud.dataflow.sdk.values.PObjectTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.SingletonPCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; import org.junit.Test; +import java.util.List; + public class WordCountTest { /** A DoFn that tokenizes lines of text into individual words. */ @@ -45,9 +49,13 @@ static class ExtractWordsFn extends DoFn { Aggregator totalWords; Aggregator maxWordLength; + PCollectionView regex; + public ExtractWordsFn(PCollectionView regex) { + this.regex = regex; + } @Override - public void startBatch(Context ctxt) { + public void startBundle(Context ctxt) { this.totalWords = ctxt.createAggregator("totalWords", new Sum.SumIntegerFn()); this.maxWordLength = ctxt.createAggregator("maxWordLength", @@ -61,62 +69,65 @@ public void processElement(ProcessContext c) { totalWords.addValue(1); if (!word.isEmpty()) { maxWordLength.addValue(word.length()); - if (Character.isLowerCase(word.charAt(0))) - c.output(word); - } else { - c.sideOutput(upper, word); + if (Character.isLowerCase(word.charAt(0))) { + c.output(word); + } else { + c.sideOutput(upper, word); + } } } } } - static TupleTag regex = new TupleTag<>(); static TupleTag upper = new TupleTag<>(); static TupleTag lower = new TupleTag<>(); static TupleTag> lowerCnts = new TupleTag<>(); static TupleTag> upperCnts = new TupleTag<>(); - public static class CountWords extends PTransform, PCollection>> { + public static class CountWords extends PTransform, PCollectionTuple> { - private final PObject regexObj; + private final PCollectionView regex; - public CountWords(PObject regexObj) { - this.regexObj = regexObj; + public CountWords(PCollectionView regex) { + this.regex = regex; } @Override - public PCollection> apply(PCollection lines) { + public PCollectionTuple apply(PCollection lines) { // Convert lines of text into individual words. - return lines - .apply(ParDo.of(new ExtractWordsFn()) - .withSideInputs(PObjectTuple.of(regex, regexObj))) - .apply(Count.create()); - //.withOutputTags(lower, TupleTagList.of(upper))); - /* - PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.create()); - PCollection> upperCounts = lowerUpper.get(upper).apply(Count.create()); + PCollectionTuple lowerUpper = lines + .apply(ParDo.of(new ExtractWordsFn(regex)) + .withSideInputs(regex) + .withOutputTags(lower, TupleTagList.of(upper))); + lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); + lowerUpper.get(upper).setCoder(StringUtf8Coder.of()); + PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.perElement()); + PCollection> upperCounts = lowerUpper.get(upper).apply(Count.perElement()); return PCollectionTuple .of(lowerCnts, lowerCounts) .and(upperCnts, upperCounts); - */ } } @Test public void testRun() throws Exception { - Pipeline p = Pipeline.create(new PipelineOptions()); - PObject regex = p.apply(CreatePObject.of("[^a-zA-Z']+")); + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection regex = p.apply(Create.of("[^a-zA-Z']+")); PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); PCollectionList list = PCollectionList.of(w1).and(w2); PCollection union = list.apply(Flatten.create()); - PCollection> lowerCounts = union.apply(new CountWords(regex)); - PObject unique = lowerCounts.apply(ApproximateUnique.>globally(16)); + PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); + PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); - System.out.println(res.get(lowerCounts)); - System.out.println(res.get(unique)); + Iterable> lower = res.get(luc.get(lowerCnts)); + Iterable> upper = res.get(luc.get(upperCnts)); + Iterable uniqCount = res.get(unique); + System.out.println(lower); + System.out.println(upper); + System.out.println(uniqCount); System.out.println(res.getAggregatorValue("totalWords", Integer.class)); System.out.println(res.getAggregatorValue("maxWordLength", Integer.class)); } From 3bd04ae9ebcd89c93e5e635d806b613c5c5438ae Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 9 Dec 2014 12:06:28 -0800 Subject: [PATCH 014/137] Dummy impls of windowing-related ProcContext functions --- .../main/java/com/cloudera/dataflow/spark/DoFnFunction.java | 6 ++++-- .../java/com/cloudera/dataflow/spark/MultiDoFnFunction.java | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) 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 index 54a2a78fe753..670f1a5c014d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.ImmutableList; import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; @@ -116,16 +117,17 @@ public DoFn.KeyedState keyedState() { @Override public void outputWithTimestamp(O output, Instant timestamp) { + output(output); } @Override public Instant timestamp() { - return null; + return Instant.now(); } @Override public Collection windows() { - return null; + return ImmutableList.of(); } } } 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 index 54383a3f8a04..8dad1fc2fe11 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -26,6 +26,7 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; @@ -127,16 +128,17 @@ public DoFn.KeyedState keyedState() { @Override public void outputWithTimestamp(O output, Instant timestamp) { + output(output); } @Override public Instant timestamp() { - return null; + return Instant.now(); } @Override public Collection windows() { - return null; + return ImmutableList.of(); } } } From ba74f19ba6bda500a9541bbd84655336f1cff149 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 9 Dec 2014 12:16:13 -0800 Subject: [PATCH 015/137] Simplify pom.xml --- runners/spark/pom.xml | 69 ------------------------------------------- 1 file changed, 69 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index fc07069846d1..f44dae835ebe 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -38,34 +38,6 @@ 1.7 - - - org.apache.felix - maven-bundle-plugin - 2.4.0 - true - - - - *;scope=compile|runtime;inline=true - - - - - - org.apache.maven.plugins - maven-dependency-plugin - 2.8 - - - copy-dependencies - package - - copy-dependencies - - - - @@ -85,49 +57,8 @@ dataflow-sdk 1.0.141206 - - com.google.apis - google-api-services-storage - v1-rev11-1.19.0 - - - com.google.apis - google-api-services-bigquery - v2-rev167-1.19.0 - - - - com.google.guava - guava-jdk5 - - - - - com.google.http-client - google-http-client-jackson2 - 1.19.0 - - - - com.fasterxml.jackson.core - jackson-core - 2.4.2 - - - com.fasterxml.jackson.core - jackson-annotations - 2.4.2 - - - org.hamcrest - hamcrest-all - 1.3 - test - junit junit From 6aa08e067d87ddb896df5a1a828bc0c526b9964f Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 9 Dec 2014 15:30:29 -0800 Subject: [PATCH 016/137] Add proper coder handling to RDD retrieval --- .../com/cloudera/dataflow/spark/CoderHelpers.java | 3 --- .../dataflow/spark/EvaluationContext.java | 15 +++++++++++++-- 2 files changed, 13 insertions(+), 5 deletions(-) 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 index 3e7bd1b20974..afe1ebdb4a57 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -18,15 +18,12 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.spark.api.java.function.Function; -import javax.annotation.Nullable; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.Collection; import java.util.List; public class CoderHelpers { 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 index f30aa809547e..6fed8c7a16af 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -28,6 +28,7 @@ import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -35,6 +36,8 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; +import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.Set; @@ -105,7 +108,7 @@ public T get(PObject value) { return (T) pobjects.get(value); } else if (rdds.containsKey(value)) { JavaRDDLike rdd = rdds.get(value); - //TODO: probably some work to do here + //TODO: need same logic from get() method below here for serialization of bytes T res = (T) Iterables.getOnlyElement(rdd.collect()); pobjects.put(value, res); return res; @@ -120,7 +123,15 @@ public T getAggregatorValue(String named, Class resultType) { @Override public Iterable get(PCollection pcollection) { - return getRDD(pcollection).collect(); + JavaRDDLike rdd = getRDD(pcollection); + final Coder coder = pcollection.getCoder(); + JavaRDDLike bytes = rdd.map(CoderHelpers.toByteFunction(coder)); + List clientBytes = bytes.collect(); + return Iterables.transform(clientBytes, new Function() { + public T apply(byte[] bytes) { + return (T) CoderHelpers.fromByteArray(bytes, coder); + } + }); } PObjectValueTuple getPObjectTuple(PTransform transform) { From 45be5086820ba71dbee81bd7785f81e38f899fe8 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 9 Dec 2014 16:35:25 -0800 Subject: [PATCH 017/137] Refactor aggregation related classes. --- .../java/com/cloudera/dataflow/spark/Agg.java | 135 ------------- .../dataflow/spark/SparkRuntimeContext.java | 23 ++- .../{ => aggregators}/AggAccumParam.java | 12 +- .../spark/aggregators/NamedAggregators.java | 178 ++++++++++++++++++ 4 files changed, 198 insertions(+), 150 deletions(-) delete mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java rename runners/spark/src/main/java/com/cloudera/dataflow/spark/{ => aggregators}/AggAccumParam.java (70%) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.java deleted file mode 100644 index 35861a003172..000000000000 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/Agg.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 com.cloudera.dataflow.spark; - -import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.common.collect.ImmutableList; - -import java.io.Serializable; -import java.util.Map; -import java.util.TreeMap; - -public class Agg implements Serializable { - - public interface State extends Serializable { - void update(VI vi); - State merge(State other); - VA current(); - VO render(); - } - - private final Map states = new TreeMap<>(); - - public Agg() { - } - - public Agg(String named, State state) { - this.states.put(named, state); - } - - public T getValue(String named, Class typeClass) { - return typeClass.cast(states.get(named).render()); - } - - public Agg merge(Agg other) { - for (Map.Entry e : other.states.entrySet()) { - State cur = states.get(e.getKey()); - if (cur == null) { - states.put(e.getKey(), e.getValue()); - } else { - states.put(e.getKey(), cur.merge(e.getValue())); - } - } - return this; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Map.Entry e : states.entrySet()) { - sb.append(e.getKey()).append(": ").append(e.getValue().render()); - } - return sb.toString(); - } - - public static class CombineState implements State { - - private Combine.CombineFn combineFn; - private VA state; - - public CombineState(Combine.CombineFn combineFn) { - this.combineFn = combineFn; - this.state = combineFn.createAccumulator(); - } - - @Override - public void update(VI vi) { - combineFn.addInput(state, vi); - } - - @Override - public State merge(State other) { - this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); - return this; - } - - @Override - public VA current() { - return state; - } - - @Override - public VO render() { - return combineFn.extractOutput(state); - } - } - - public static class SerState implements State { - - private SerializableFunction, VO> sfunc; - private VO state; - - public SerState(SerializableFunction, VO> sfunc) { - this.sfunc = sfunc; - this.state = sfunc.apply(ImmutableList.of()); - } - - @Override - public void update(VI vi) { - this.state = sfunc.apply(ImmutableList.of(vi, (VI) state)); - } - - @Override - public State merge(State other) { - this.state = sfunc.apply(ImmutableList.of((VI) state, (VI) other.current())); - return this; - } - - @Override - public VO current() { - return state; - } - - @Override - public VO render() { - return state; - } - } - -} 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 index fb405f08795d..bbbbf400fa47 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -17,6 +17,8 @@ */ package com.cloudera.dataflow.spark; +import com.cloudera.dataflow.spark.aggregators.AggAccumParam; +import com.cloudera.dataflow.spark.aggregators.NamedAggregators; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; @@ -29,13 +31,15 @@ import java.util.HashMap; import java.util.Map; +/** + * The SparkRuntimeContext exposes + */ class SparkRuntimeContext implements Serializable { - - private Accumulator accum; + private Accumulator accum; private Map aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.accum = jsc.accumulator(new Agg(), new AggAccumParam()); + this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); } public T getAggregatorValue(String named, Class typeClass) { @@ -51,8 +55,8 @@ public synchronized Aggregator createAggregator( SerializableFunction, AO> sfunc) { Aggregator aggregator = aggregators.get(named); if (aggregator == null) { - Agg.SerState state = new Agg.SerState<>(sfunc); - accum.add(new Agg(named, state)); + NamedAggregators.SerFunctionState state = new NamedAggregators.SerFunctionState<>(sfunc); + accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator(state); aggregators.put(named, aggregator); } @@ -64,8 +68,9 @@ public synchronized Aggregator createAggregator( Combine.CombineFn combineFn) { Aggregator aggregator = aggregators.get(named); if (aggregator == null) { - Agg.CombineState state = new Agg.CombineState<>(combineFn); - accum.add(new Agg(named, state)); + NamedAggregators.CombineFunctionState state = new NamedAggregators + .CombineFunctionState<>(combineFn); + accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator(state); aggregators.put(named, aggregator); } @@ -73,9 +78,9 @@ public synchronized Aggregator createAggregator( } private static class SparkAggregator implements Aggregator { - private final Agg.State state; + private final NamedAggregators.State state; - public SparkAggregator(Agg.State state) { + public SparkAggregator(NamedAggregators.State state) { this.state = state; } diff --git a/runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java b/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java similarity index 70% rename from runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java rename to runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java index 221ad03e95e1..91b883798906 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/AggAccumParam.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/AggAccumParam.java @@ -15,23 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.cloudera.dataflow.spark; +package com.cloudera.dataflow.spark.aggregators; import org.apache.spark.AccumulatorParam; -public class AggAccumParam implements AccumulatorParam { +public class AggAccumParam implements AccumulatorParam { @Override - public Agg addAccumulator(Agg current, Agg added) { + public NamedAggregators addAccumulator(NamedAggregators current, NamedAggregators added) { return current.merge(added); } @Override - public Agg addInPlace(Agg current, Agg added) { + public NamedAggregators addInPlace(NamedAggregators current, NamedAggregators added) { return addAccumulator(current, added); } @Override - public Agg zero(Agg initialValue) { - return new Agg(); + public NamedAggregators zero(NamedAggregators initialValue) { + return new NamedAggregators(); } } 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 new file mode 100644 index 000000000000..bca0ac9cbb14 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/aggregators/NamedAggregators.java @@ -0,0 +1,178 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 com.cloudera.dataflow.spark.aggregators; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.common.collect.ImmutableList; + +import java.io.Serializable; +import java.util.Map; +import java.util.TreeMap; + +/** + * What is an Agg? + * wrapper around a map of named aggregators. + * This allows us to add a named aggregator on the fly. + * we create an accumulable instance of aggs every time in the spark context. + * When the dataflow + */ +public class NamedAggregators implements Serializable { + /** + * + * @param Input data type + * @param Intermediate data type (useful for averages) + * @param Output datatype + */ + public interface State extends Serializable { + /** + * + * @param element + */ + void update(In element); + State merge(State other); + Inter current(); + Out render(); + } + + /** + * Why is this final if you later add states to it? + */ + private final Map mNamedAggregators = new TreeMap<>(); + + public NamedAggregators() { + } + + /** + * is "named" the label for a state? + * @param named + * @param state + */ + public NamedAggregators(String named, State state) { + this.mNamedAggregators.put(named, state); + } + + /** + * + * @param named + * @param typeClass + * @param + * @return + */ + public T getValue(String named, Class typeClass) { + return typeClass.cast(mNamedAggregators.get(named).render()); + } + + public NamedAggregators merge(NamedAggregators other) { + for (Map.Entry e : other.mNamedAggregators.entrySet()) { + State cur = mNamedAggregators.get(e.getKey()); + if (cur == null) { + mNamedAggregators.put(e.getKey(), e.getValue()); + } else { + mNamedAggregators.put(e.getKey(), cur.merge(e.getValue())); + } + } + return this; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (Map.Entry e : mNamedAggregators.entrySet()) { + sb.append(e.getKey()).append(": ").append(e.getValue().render()); + } + return sb.toString(); + } + + /** + * => combineFunction in data flow. + * @param + * @param + * @param + */ + public static class CombineFunctionState implements State { + + private Combine.CombineFn combineFn; + private Inter state; + + public CombineFunctionState(Combine.CombineFn combineFn) { + this.combineFn = combineFn; + this.state = combineFn.createAccumulator(); + } + + @Override + public void update(In element) { + combineFn.addInput(state, element); + } + + @Override + public State merge(State 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); + } + } + + /** + * states correspond to dataflow objects. this one => seriazable function + * @param + * @param + */ + public static class SerFunctionState implements State { + + private SerializableFunction, Out> sfunc; + private Out state; + + public SerFunctionState(SerializableFunction, Out> sfunc) { + this.sfunc = sfunc; + this.state = sfunc.apply(ImmutableList.of()); + } + + @Override + public void update(In element) { + this.state = sfunc.apply(ImmutableList.of(element, (In) state)); + } + + @Override + public State merge(State other) { + // Add exception catching and logging here. + this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); + return this; + } + + @Override + public Out current() { + return state; + } + + @Override + public Out render() { + return state; + } + } + +} From 67cf364252fcf1df66ad97dac12d0cc006aa5d46 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Thu, 11 Dec 2014 12:12:45 -0800 Subject: [PATCH 018/137] Add README.md and update project version in pom.xml. --- runners/spark/README.md | 17 +++++++++++++++++ runners/spark/pom.xml | 17 +---------------- 2 files changed, 18 insertions(+), 16 deletions(-) create mode 100644 runners/spark/README.md diff --git a/runners/spark/README.md b/runners/spark/README.md new file mode 100644 index 000000000000..0921f4acd523 --- /dev/null +++ b/runners/spark/README.md @@ -0,0 +1,17 @@ +spark-dataflow +============== +Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing a pipeline on a spark cluster is easy: Depend on spark-dataflow in your project +and execute your pipeline in a program by calling `SparkPipelineRunner.run`. + +The Maven coordinates of the current version of this project are: + com.cloudera.dataflow.spark + dataflow-spark + 0.0.1 + +An example of running a pipeline against a spark cluster in local mode with 2 +threads. + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + /** logic for building your pipeline */ + EvaluationResult result = new SparkPipelineRunner("local[2]").run() + + diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index f44dae835ebe..d02bdee4f574 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -1,19 +1,4 @@ - @@ -21,7 +6,7 @@ Dataflow on Spark com.cloudera.dataflow.spark dataflow-spark - 0.1.0 + 0.0.1 jar From 137d54a1efe4e134cc7645c01eb013db10d2be30 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 9 Dec 2014 16:37:46 -0800 Subject: [PATCH 019/137] Adds Javadoc and Tests to project. --- .../cloudera/dataflow/spark/CoderHelpers.java | 39 +++++++++++ .../cloudera/dataflow/spark/DoFnFunction.java | 25 ++++++- .../dataflow/spark/EvaluationContext.java | 4 +- .../dataflow/spark/EvaluationResult.java | 30 ++++++++- .../dataflow/spark/MultiDoFnFunction.java | 10 +++ .../dataflow/spark/SparkPipelineRunner.java | 34 +++++++--- .../dataflow/spark/SparkRuntimeContext.java | 67 ++++++++++++++----- .../dataflow/spark/WordCountTest.java | 20 +++--- 8 files changed, 190 insertions(+), 39 deletions(-) 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 index afe1ebdb4a57..4a1932938c7d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -26,7 +26,17 @@ import java.io.IOException; import java.util.List; +/** + * Serialization utility class. + */ public class CoderHelpers { + /** + * Utility method for serializing an object using the specified coder. + * + * @param value Value to serialize. + * @param coder Coder to serialize with. + * @return Byte array representing serialized object. + */ static byte[] toByteArray(Object value, Coder coder) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); try { @@ -37,6 +47,13 @@ static byte[] toByteArray(Object value, Coder coder) { 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. + * @return List of bytes representing serialized objects. + */ static List toByteArrays(Iterable values, final Coder coder) { List res = Lists.newLinkedList(); for (Object value : values) { @@ -45,6 +62,14 @@ static List toByteArrays(Iterable values, final Coder 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 Type of object to be returned. + * @return Deserialized object. + */ static T fromByteArray(byte[] serialized, Coder coder) { ByteArrayInputStream bais = new ByteArrayInputStream(serialized); try { @@ -54,6 +79,13 @@ static T fromByteArray(byte[] serialized, Coder coder) { } } + /** + * A function wrapper for converting an object to a bytearray. + * + * @param coder Coder to serialize with. + * @param The type of the object being serialized. + * @return A function that accepts an object and returns its coder-serialized form. + */ static Function toByteFunction(final Coder coder) { return new Function() { @Override @@ -63,6 +95,13 @@ public byte[] call(T t) throws Exception { }; } + /** + * A function wrapper for converting a byte array to an object. + * + * @param coder Coder to deserialize with. + * @param The type of the object being deserialized. + * @return A function that accepts a byte array and returns its corresponding object. + */ static Function fromByteFunction(final Coder coder) { return new Function() { @Override 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 index 670f1a5c014d..6ab1fa154ce4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -34,13 +34,27 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.logging.Logger; +/** + * Dataflow's Do functions correspond to Spark's FlatMap functions. + * + * @param Input element type. + * @param Output element type. + */ class DoFnFunction implements FlatMapFunction, O> { + private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); private final DoFn fn; private final SparkRuntimeContext runtimeContext; private final Map, BroadcastHelper> sideInputs; + /** + * + * @param fn DoFunction to be wrapped. + * @param runtime Runtime to apply function in. + * @param sideInputs Side inputs used in DoFunction. + */ public DoFnFunction( DoFn fn, SparkRuntimeContext runtime, @@ -50,14 +64,18 @@ public DoFnFunction( this.sideInputs = sideInputs; } + @Override public Iterable call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(fn); + ProcCtxt ctxt = new ProcCtxt<>(fn); + //setup fn.startBundle(ctxt); + //operation while (iter.hasNext()) { ctxt.element = iter.next(); fn.processElement(ctxt); } + //cleanup fn.finishBundle(ctxt); return ctxt.outputs; } @@ -88,7 +106,10 @@ public synchronized void output(O o) { @Override public void sideOutput(TupleTag tupleTag, T t) { - // A no-op in this context; maybe add some logging + LOG.warning("sideoutput is an unsupported operation for DoFnFunctions. Use a " + + "MultiDoFunction"); + throw new UnsupportedOperationException("sideOutput is an unsupported operation for " + + "doFunctions, use a MultiDoFunction instead."); } @Override 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 index 6fed8c7a16af..38a94c7a20f3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -36,11 +36,13 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Set; +/** + * Evaluation context allows us to define how pipeline instructions + */ public class EvaluationContext implements EvaluationResult { private final JavaSparkContext jsc; private final Pipeline pipeline; 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 index 74fe20549fc1..577f7ce28e3d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -21,8 +21,36 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PObject; +/** + * Interfacing for retrieve the result(s) of running a pipeline. Allows us to translate between + * PObjects or PCollections and T's or coolections of T's. + */ public interface EvaluationResult extends PipelineResult { + /** + * Retrieves an iterable of results associated with the PCollection passed in. + * + * @param pcollection Collection we wish to translate. + * @param Type of elements contained in collection. + * @return Natively types result associated with collection. + */ Iterable get(PCollection pcollection); + + /** + * Retrieve an object of Type T associated with the PObject passed in. + * + * @param pobject PObject we wish to translate. + * @param Type of object to return. + * @return Native object. + */ T get(PObject pobject); - T getAggregatorValue(String named, Class resultType); + + /** + * Retrieves the final value of the aggregator. + * + * @param aggName name of aggragtor. + * @param resultType Class of final result of aggregatiohn. + * @param Type of final result of aggregation. + * @return Result of aggregation associated with specified name. + */ + T getAggregatorValue(String aggName, Class resultType); } 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 index 8dad1fc2fe11..6ef6b82c7263 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -38,7 +38,17 @@ import java.util.Iterator; import java.util.Map; +/** + * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enrishing the + * undelrying data with multiple TupleTags. + * + * @param Input type for DoFunction. + * @param Output type for DoFunction. + */ class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { + // TODO: I think implementing decoding logic will allow us to do away with having two types of + // DoFunctions. Josh originally made these two classes in order to help ease the typing of + // results. Correctly using coders should just fix this. private final DoFn fn; private final SparkRuntimeContext runtimeContext; 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 index e995c2d302e0..2af41768b7bb 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -59,23 +59,40 @@ import java.util.Map; import java.util.logging.Logger; +/** + * 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 in Spark's local mode with two threads, we would do the following: + * Pipeline p = + * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); + */ public class SparkPipelineRunner extends PipelineRunner { private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); - + /** The url of the spark master to connect to. */ private final String master; + /** + * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single + * thread. + */ public SparkPipelineRunner() { this("local"); } + /** + * Constructor for a pipeline runner. + * + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). + */ public SparkPipelineRunner(String master) { this.master = Preconditions.checkNotNull(master); } @Override public EvaluationResult run(Pipeline pipeline) { + // TODO: get master from options JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); @@ -104,6 +121,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { @Override public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); TransformEvaluator evaluator = EVALUATORS.get(transform.getClass()); if (evaluator == null) { @@ -148,7 +166,6 @@ public T get(String fieldname, Object value) { public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); JavaRDD rdd = context.getSparkContext().textFile(pattern); - // TODO: handle coders context.setOutputRDD(transform, rdd); } }; @@ -157,7 +174,6 @@ public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { JavaRDDLike last = context.getInputRDD(transform); - // TODO: handle coders String pattern = transform.getFilenamePrefix(); last.saveAsTextFile(pattern); } @@ -168,7 +184,6 @@ public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); JavaRDD rdd = context.getSparkContext().textFile(pattern); - // TODO: handle coders context.setOutputRDD(transform, rdd); } }; @@ -177,10 +192,6 @@ public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { @Override public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { JavaRDDLike last = context.getInputRDD(transform); - Coder coder = null; - if (coder != null) { - //TODO - } String pattern = transform.getFilenamePrefix(); last.saveAsTextFile(pattern); } @@ -212,7 +223,9 @@ public void evaluate(Convert.ToIterable transform, EvaluationContext context) { context.setPObjectValue(out, context.get(in)); } }; - + /** + * needs to handle coders + */ private static TransformEvaluator TO_ITER_WIN = new TransformEvaluator() { @Override @@ -359,6 +372,9 @@ public static void registerEvaluator( EVALUATORS.put(transformClass, evaluator); } + /** + * helps map from the functions being applied to transform evaluations + */ private static final Map, TransformEvaluator> EVALUATORS = Maps.newHashMap(); static { registerEvaluator(TextIO.Read.Bound.class, READ_TEXT); 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 index bbbbf400fa47..9fc3f72fee47 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -32,43 +32,73 @@ import java.util.Map; /** - * The SparkRuntimeContext exposes + * The SparkRuntimeContext allows us to define useful features on the client side before our + * data flow program is launched. */ class SparkRuntimeContext implements Serializable { + /** An accumulator that is a map from names to aggregators. */ private Accumulator accum; + /** Map fo names to dataflow aggregators. */ private Map aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); } - public T getAggregatorValue(String named, Class typeClass) { - return accum.value().getValue(named, typeClass); + /** + * 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 Type of object to be returned. + * @return The value of the aggregator. + */ + public T getAggregatorValue(String aggregatorName, Class typeClass) { + return accum.value().getValue(aggregatorName, typeClass); } public synchronized PipelineOptions getPipelineOptions() { - return null; // TODO + //TODO: Support this. + throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); } - public synchronized Aggregator createAggregator( + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param sfunc Serializable function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( String named, - SerializableFunction, AO> sfunc) { + SerializableFunction, Out> sfunc) { Aggregator aggregator = aggregators.get(named); if (aggregator == null) { - NamedAggregators.SerFunctionState state = new NamedAggregators.SerFunctionState<>(sfunc); + NamedAggregators.SerFunctionState state = new NamedAggregators + .SerFunctionState<>(sfunc); accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator(state); aggregators.put(named, aggregator); } return aggregator; } - - public synchronized Aggregator createAggregator( + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param combineFn Combine function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( String named, - Combine.CombineFn combineFn) { + Combine.CombineFn combineFn) { Aggregator aggregator = aggregators.get(named); if (aggregator == null) { - NamedAggregators.CombineFunctionState state = new NamedAggregators + NamedAggregators.CombineFunctionState state = new NamedAggregators .CombineFunctionState<>(combineFn); accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator(state); @@ -77,16 +107,21 @@ public synchronized Aggregator createAggregator( return aggregator; } - private static class SparkAggregator implements Aggregator { - private final NamedAggregators.State state; + /** + * Initialize spark aggregators exactly once. + * + * @param Type of element fed in to aggregator. + */ + private static class SparkAggregator implements Aggregator { + private final NamedAggregators.State state; - public SparkAggregator(NamedAggregators.State state) { + public SparkAggregator(NamedAggregators.State state) { this.state = state; } @Override - public void addValue(VI vi) { - state.update(vi); + public void addValue(In elem) { + state.update(elem); } } } diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index a04b41ea34fe..3fd69db27d6f 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -38,10 +38,9 @@ import com.google.cloud.dataflow.sdk.values.SingletonPCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.cloud.dataflow.sdk.values.TupleTagList; +import org.junit.Assert; import org.junit.Test; -import java.util.List; - public class WordCountTest { /** A DoFn that tokenizes lines of text into individual words. */ @@ -122,13 +121,14 @@ public void testRun() throws Exception { PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); - Iterable> lower = res.get(luc.get(lowerCnts)); - Iterable> upper = res.get(luc.get(upperCnts)); - Iterable uniqCount = res.get(unique); - System.out.println(lower); - System.out.println(upper); - System.out.println(uniqCount); - System.out.println(res.getAggregatorValue("totalWords", Integer.class)); - System.out.println(res.getAggregatorValue("maxWordLength", Integer.class)); + Iterable> actualLower = res.get(luc.get(lowerCnts)); + Iterable> actualUpper = res.get(luc.get(upperCnts)); + Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); + Iterable 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); } } From b9545895738856693f3b5a40e127e348763a91f8 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Thu, 11 Dec 2014 14:28:00 -0800 Subject: [PATCH 020/137] Add apache2 license and cloudera copyright. --- runners/spark/LICENSE | 161 ++++++++++++++++++ runners/spark/pom.xml | 11 ++ .../dataflow/spark/BroadcastHelper.java | 25 ++- .../cloudera/dataflow/spark/CoderHelpers.java | 25 ++- .../cloudera/dataflow/spark/DoFnFunction.java | 26 ++- .../dataflow/spark/EvaluationContext.java | 25 ++- .../dataflow/spark/EvaluationResult.java | 26 ++- .../dataflow/spark/MultiDoFnFunction.java | 25 ++- .../dataflow/spark/SparkPipelineRunner.java | 26 ++- .../dataflow/spark/SparkRuntimeContext.java | 25 ++- .../dataflow/spark/TransformEvaluator.java | 26 ++- .../spark/aggregators/AggAccumParam.java | 26 ++- .../spark/aggregators/NamedAggregators.java | 26 ++- .../dataflow/spark/WordCountTest.java | 26 ++- 14 files changed, 311 insertions(+), 168 deletions(-) create mode 100644 runners/spark/LICENSE diff --git a/runners/spark/LICENSE b/runners/spark/LICENSE new file mode 100644 index 000000000000..a647a925d538 --- /dev/null +++ b/runners/spark/LICENSE @@ -0,0 +1,161 @@ +Apache License + +Version 2.0, January 2004 + +http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + +"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document. + +"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License. + +"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. + +"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License. + +"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files. + +"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types. + +"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below). + +"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof. + +"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution." + +"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions: + +You must give any other recipients of the Work or Derivative Works a copy of this License; and + +You must cause any modified files to carry prominent notices stating that You changed the files; and + +You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and + +If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work +To apply the Apache License to your work, attach the following boilerplate notice, with the fields enclosed by brackets "[]" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a file or class name and description of purpose be included on the same "printed page" as the copyright notice for easier identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this 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. + + +=============================================================================== + +This project also distributes third-party software licensed under other open-source licenses: + +BSD License (http://opensource.org/licenses/BSD-3-Clause): + + JPMML (https://github.com/jpmml/jpmml-model) + ParaNamer Core (https://github.com/paul-hammant/paranamer) + Protocol Buffer Java API (http://code.google.com/p/protobuf) + +MIT License (http://opensource.org/licenses/mit-license.php): + + SLF4J (http://www.slf4j.org/) +Apache License + +Version 2.0, January 2004 + +http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + +"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document. + +"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License. + +"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. + +"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License. + +"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files. + +"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types. + +"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below). + +"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof. + +"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution." + +"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions: + +You must give any other recipients of the Work or Derivative Works a copy of this License; and + +You must cause any modified files to carry prominent notices stating that You changed the files; and + +You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and + +If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work +To apply the Apache License to your work, attach the following boilerplate notice, with the fields enclosed by brackets "[]" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a file or class name and description of purpose be included on the same "printed page" as the copyright notice for easier identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this 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. + + diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index d02bdee4f574..8a5a6d379882 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -1,4 +1,15 @@ + 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 index 48422e469283..de3afa131c19 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -1,19 +1,16 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Cloudera, Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"). You may not use this file except in + * compliance with the License. You may obtain a copy of the License at * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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://www.apache.org/licenses/LICENSE-2.0 + * + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 4a1932938c7d..66b6f4c9c015 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -1,19 +1,16 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 6ab1fa154ce4..0ce63a2fcaac 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 38a94c7a20f3..dbee7ef17fcb 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -1,19 +1,16 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 577f7ce28e3d..84dec7780aab 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 6ef6b82c7263..8200396ddf64 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -1,19 +1,16 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 2af41768b7bb..6536804e876a 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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.util.Maps; 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 index 9fc3f72fee47..2af94b18a7a2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -1,19 +1,16 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 3147a1be7c65..f86533203389 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Cloudera, Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"). You may not use this file except in + * compliance with the License. You may obtain a copy of the License at * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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://www.apache.org/licenses/LICENSE-2.0 + * + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index 91b883798906..a3055a256ded 100644 --- 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 @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Cloudera, Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"). You may not use this file except in + * compliance with the License. You may obtain a copy of the License at * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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://www.apache.org/licenses/LICENSE-2.0 + * + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; 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 index bca0ac9cbb14..61cd9278a9fa 100644 --- 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 @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * 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 + * 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. + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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 com.google.cloud.dataflow.sdk.transforms.Combine; diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 3fd69db27d6f..81cd36b9e5e8 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -1,20 +1,18 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at +/* + * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Cloudera, Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"). You may not use this file except in + * compliance with the License. You may obtain a copy of the License at * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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://www.apache.org/licenses/LICENSE-2.0 + * + * This software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 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; From 1523ffdae2a2fdf8029dfab68da1430edc6fe951 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Thu, 11 Dec 2014 16:40:28 -0800 Subject: [PATCH 021/137] Adds custom checkstyle. more checkstyle improvements --- runners/spark/build-resources/checkstyle.xml | 208 +++++++ runners/spark/build-resources/header-file.txt | 14 + runners/spark/pom.xml | 114 ++-- .../dataflow/spark/BroadcastHelper.java | 42 +- .../cloudera/dataflow/spark/CoderHelpers.java | 154 ++--- .../cloudera/dataflow/spark/DoFnFunction.java | 215 ++++--- .../dataflow/spark/EvaluationContext.java | 228 +++---- .../dataflow/spark/EvaluationResult.java | 14 +- .../dataflow/spark/MultiDoFnFunction.java | 206 +++---- .../dataflow/spark/SparkPipelineRunner.java | 568 +++++++++--------- .../dataflow/spark/SparkRuntimeContext.java | 172 +++--- .../dataflow/spark/TransformEvaluator.java | 6 +- .../spark/aggregators/AggAccumParam.java | 24 +- .../spark/aggregators/NamedAggregators.java | 223 +++---- .../dataflow/spark/WordCountTest.java | 157 ++--- 15 files changed, 1297 insertions(+), 1048 deletions(-) create mode 100644 runners/spark/build-resources/checkstyle.xml create mode 100644 runners/spark/build-resources/header-file.txt diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml new file mode 100644 index 000000000000..4b73e54aaca3 --- /dev/null +++ b/runners/spark/build-resources/checkstyle.xml @@ -0,0 +1,208 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/runners/spark/build-resources/header-file.txt b/runners/spark/build-resources/header-file.txt new file mode 100644 index 000000000000..19f441881819 --- /dev/null +++ b/runners/spark/build-resources/header-file.txt @@ -0,0 +1,14 @@ +/* + * 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. + */ \ No newline at end of file diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 8a5a6d379882..3d4afdf938f0 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -10,56 +10,76 @@ CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - - 4.0.0 - Dataflow on Spark - com.cloudera.dataflow.spark - dataflow-spark - 0.0.1 - jar + 4.0.0 + Dataflow on Spark + com.cloudera.dataflow.spark + dataflow-spark + 0.0.1 + jar - - UTF-8 - + + UTF-8 + - - - - maven-compiler-plugin - 3.1 - - 1.7 - 1.7 - - - - + + + + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.13 + + build-resources/header-file.txt + build-resources/checkstyle.xml + + + + - - - org.apache.spark - spark-core_2.10 - 1.1.0 - - - com.google.guava - guava - 18.0 - - - com.google.cloud.dataflow - dataflow-sdk - 1.0.141206 - + + + org.apache.spark + spark-core_2.10 + 1.1.0 + + + com.google.guava + guava + 18.0 + + + com.google.cloud.dataflow + dataflow-sdk + 1.0.141206 + - - - junit - junit - 4.11 - test - - + + + junit + junit + 4.11 + test + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.13 + + + + 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 index de3afa131c19..f0ee6ad9e4e4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -14,35 +14,35 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.coders.Coder; -import org.apache.spark.broadcast.Broadcast; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.Serializable; +import com.google.cloud.dataflow.sdk.coders.Coder; +import org.apache.spark.broadcast.Broadcast; + class BroadcastHelper implements Serializable { - private final Broadcast bcast; - private final Coder coder; - private transient T value; + private final Broadcast bcast; + private final Coder coder; + private transient T value; - BroadcastHelper(Broadcast bcast, Coder coder) { - this.bcast = bcast; - this.coder = coder; - } + BroadcastHelper(Broadcast bcast, Coder coder) { + this.bcast = bcast; + this.coder = coder; + } - public synchronized T getValue() { - if (value == null) { - value = deserialize(); + public synchronized T getValue() { + if (value == null) { + value = deserialize(); + } + return value; } - return value; - } - private T deserialize() { - try { - return coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); - } catch (IOException e) { - throw new RuntimeException("Error deserializing broadcast variable", e); + private T deserialize() { + try { + return coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error deserializing broadcast variable", e); + } } - } } 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 index 66b6f4c9c015..03f02f4adb68 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -14,97 +14,97 @@ */ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.common.collect.Lists; -import org.apache.spark.api.java.function.Function; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.List; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.common.collect.Lists; +import org.apache.spark.api.java.function.Function; + /** * Serialization utility class. */ public class CoderHelpers { - /** - * Utility method for serializing an object using the specified coder. - * - * @param value Value to serialize. - * @param coder Coder to serialize with. - * @return Byte array representing serialized object. - */ - static byte[] toByteArray(Object value, Coder coder) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - coder.encode(value, baos, new Coder.Context(true)); - } catch (IOException e) { - throw new RuntimeException("Error encoding value: " + value, e); + /** + * Utility method for serializing an object using the specified coder. + * + * @param value Value to serialize. + * @param coder Coder to serialize with. + * @return Byte array representing serialized object. + */ + static byte[] toByteArray(Object value, Coder coder) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + coder.encode(value, baos, new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error encoding value: " + value, e); + } + return baos.toByteArray(); } - 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. - * @return List of bytes representing serialized objects. - */ - static List toByteArrays(Iterable values, final Coder coder) { - List res = Lists.newLinkedList(); - for (Object value : values) { - res.add(toByteArray(value, coder)); + /** + * Utility method for serializing a Iterable of values using the specified coder. + * + * @param values Values to serialize. + * @param coder Coder to serialize with. + * @return List of bytes representing serialized objects. + */ + static List toByteArrays(Iterable values, final Coder coder) { + List res = Lists.newLinkedList(); + for (Object value : values) { + res.add(toByteArray(value, coder)); + } + return res; } - 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 Type of object to be returned. - * @return Deserialized object. - */ - static T fromByteArray(byte[] serialized, Coder coder) { - ByteArrayInputStream bais = new ByteArrayInputStream(serialized); - try { - return coder.decode(bais, new Coder.Context(true)); - } catch (IOException e) { - throw new RuntimeException("Error decoding bytes for coder: " + coder, e); + /** + * Utility method for deserializing a byte array using the specified coder. + * + * @param serialized bytearray to be deserialized. + * @param coder Coder to deserialize with. + * @param Type of object to be returned. + * @return Deserialized object. + */ + static T fromByteArray(byte[] serialized, Coder coder) { + ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + try { + return coder.decode(bais, new Coder.Context(true)); + } catch (IOException e) { + throw new RuntimeException("Error decoding bytes for coder: " + coder, e); + } } - } - /** - * A function wrapper for converting an object to a bytearray. - * - * @param coder Coder to serialize with. - * @param The type of the object being serialized. - * @return A function that accepts an object and returns its coder-serialized form. - */ - static Function toByteFunction(final Coder coder) { - return new Function() { - @Override - public byte[] call(T t) throws Exception { - return toByteArray(t, coder); - } - }; - } + /** + * A function wrapper for converting an object to a bytearray. + * + * @param coder Coder to serialize with. + * @param The type of the object being serialized. + * @return A function that accepts an object and returns its coder-serialized form. + */ + static Function toByteFunction(final Coder coder) { + return new Function() { + @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 The type of the object being deserialized. - * @return A function that accepts a byte array and returns its corresponding object. - */ - static Function fromByteFunction(final Coder coder) { - return new Function() { - @Override - public T call(byte[] bytes) throws Exception { - return fromByteArray(bytes, coder); - } - }; - } + /** + * A function wrapper for converting a byte array to an object. + * + * @param coder Coder to deserialize with. + * @param The type of the object being deserialized. + * @return A function that accepts a byte array and returns its corresponding object. + */ + static Function fromByteFunction(final Coder coder) { + return new Function() { + @Override + public T call(byte[] bytes) throws Exception { + return fromByteArray(bytes, coder); + } + }; + } } 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 index 0ce63a2fcaac..983a5959b4be 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -15,6 +15,13 @@ package com.cloudera.dataflow.spark; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.logging.Logger; + import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -27,13 +34,6 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.logging.Logger; - /** * Dataflow's Do functions correspond to Spark's FlatMap functions. * @@ -41,112 +41,111 @@ * @param Output element type. */ class DoFnFunction implements FlatMapFunction, O> { - private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); - - private final DoFn fn; - private final SparkRuntimeContext runtimeContext; - private final Map, BroadcastHelper> sideInputs; - - /** - * - * @param fn DoFunction to be wrapped. - * @param runtime Runtime to apply function in. - * @param sideInputs Side inputs used in DoFunction. - */ - public DoFnFunction( - DoFn fn, - SparkRuntimeContext runtime, - Map, BroadcastHelper> sideInputs) { - this.fn = fn; - this.runtimeContext = runtime; - this.sideInputs = sideInputs; - } - - - @Override - public Iterable call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt<>(fn); - //setup - fn.startBundle(ctxt); - //operation - while (iter.hasNext()) { - ctxt.element = iter.next(); - fn.processElement(ctxt); - } - //cleanup - fn.finishBundle(ctxt); - return ctxt.outputs; - } - - private class ProcCtxt extends DoFn.ProcessContext { - - private List outputs = new LinkedList<>(); - private I element; - - public ProcCtxt(DoFn fn) { - fn.super(); + private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); + + private final DoFn mFunction; + private final SparkRuntimeContext mRuntimeContext; + private final Map, 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 fn, + SparkRuntimeContext runtime, + Map, BroadcastHelper> sideInputs) { + this.mFunction = fn; + this.mRuntimeContext = runtime; + this.mSideInputs = sideInputs; } - @Override - public PipelineOptions getPipelineOptions() { - return runtimeContext.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - return (T) sideInputs.get(view.getTagInternal()).getValue(); - } @Override - public synchronized void output(O o) { - outputs.add(o); + public Iterable call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt<>(mFunction); + //setup + mFunction.startBundle(ctxt); + //operation + while (iter.hasNext()) { + ctxt.element = iter.next(); + mFunction.processElement(ctxt); + } + //cleanup + mFunction.finishBundle(ctxt); + return ctxt.outputs; } - @Override - public void sideOutput(TupleTag tupleTag, T t) { - LOG.warning("sideoutput is an unsupported operation for DoFnFunctions. Use a " + - "MultiDoFunction"); - throw new UnsupportedOperationException("sideOutput is an unsupported operation for " + - "doFunctions, use a MultiDoFunction instead."); - } - - @Override - public Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - return runtimeContext.createAggregator(named, combineFn); - } - - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return runtimeContext.createAggregator(named, sfunc); - } - - @Override - public I element() { - return element; - } - - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public Collection windows() { - return ImmutableList.of(); + private class ProcCtxt extends DoFn.ProcessContext { + + private List outputs = new LinkedList<>(); + private I element; + + public ProcCtxt(DoFn fn) { + fn.super(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return mRuntimeContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return (T) mSideInputs.get(view.getTagInternal()).getValue(); + } + + @Override + public synchronized void output(O o) { + outputs.add(o); + } + + @Override + public void sideOutput(TupleTag tupleTag, T t) { + LOG.warning("sideoutput is an unsupported operation for DoFnFunctions. Use a " + + "MultiDoFunction"); + throw new UnsupportedOperationException("sideOutput is an unsupported operation for " + + "doFunctions, use a MultiDoFunction instead."); + } + + @Override + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return mRuntimeContext.createAggregator(named, combineFn); + } + + @Override + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return mRuntimeContext.createAggregator(named, sfunc); + } + + @Override + public I element() { + return element; + } + + @Override + public DoFn.KeyedState keyedState() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + output(output); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Collection windows() { + return ImmutableList.of(); + } } - } } 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 index dbee7ef17fcb..ae26ee9d9d4b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -14,6 +14,10 @@ */ package com.cloudera.dataflow.spark; +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.transforms.PTransform; @@ -33,119 +37,121 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * 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 rdds = Maps.newHashMap(); - private final Set multireads = Sets.newHashSet(); - private final Map pobjects = Maps.newHashMap(); - - public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { - this.jsc = jsc; - this.pipeline = pipeline; - this.runtime = new SparkRuntimeContext(jsc, pipeline); - } - - JavaSparkContext getSparkContext() { - return jsc; - } - Pipeline getPipeline() { return pipeline; } - SparkRuntimeContext getRuntimeContext() { return runtime; } - - I getInput(PTransform transform) { - return (I) pipeline.getInput(transform); - } - - O getOutput(PTransform transform) { - return (O) pipeline.getOutput(transform); - } - - void setOutputRDD(PTransform transform, JavaRDDLike rdd) { - rdds.put((PValue) getOutput(transform), rdd); - } - - void setPObjectValue(PObject pobject, Object value) { - pobjects.put(pobject, value); - } - - JavaRDDLike getRDD(PValue pvalue) { - JavaRDDLike rdd = rdds.get(pvalue); - if (multireads.contains(pvalue)) { - // Ensure the RDD is marked as cached - rdd.rdd().cache(); - } else { - multireads.add(pvalue); - } - return rdd; - } - - void setRDD(PValue pvalue, JavaRDDLike rdd) { - rdds.put(pvalue, rdd); - } - - JavaRDDLike getInputRDD(PTransform transform) { - return getRDD((PValue) pipeline.getInput(transform)); - } - - BroadcastHelper getBroadcastHelper(PObject value) { - Coder coder = value.getCoder(); - Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(get(value), coder)); - return new BroadcastHelper<>(bcast, coder); - } - - @Override - public T get(PObject value) { - if (pobjects.containsKey(value)) { - return (T) pobjects.get(value); - } else if (rdds.containsKey(value)) { - JavaRDDLike rdd = rdds.get(value); - //TODO: need same logic from get() method below here for serialization of bytes - 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 getAggregatorValue(String named, Class resultType) { - return runtime.getAggregatorValue(named, resultType); - } - - @Override - public Iterable get(PCollection pcollection) { - JavaRDDLike rdd = getRDD(pcollection); - final Coder coder = pcollection.getCoder(); - JavaRDDLike bytes = rdd.map(CoderHelpers.toByteFunction(coder)); - List clientBytes = bytes.collect(); - return Iterables.transform(clientBytes, new Function() { - public T apply(byte[] bytes) { - return (T) CoderHelpers.fromByteArray(bytes, coder); - } - }); - } - - PObjectValueTuple getPObjectTuple(PTransform transform) { - PObjectTuple pot = (PObjectTuple) pipeline.getInput(transform); - PObjectValueTuple povt = PObjectValueTuple.empty(); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - povt = povt.and((TupleTag) e.getKey(), get(e.getValue())); - } - return povt; - } - - void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { - PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - pobjects.put(e.getValue(), outputValues.get(e.getKey())); - } - } + private final JavaSparkContext jsc; + private final Pipeline pipeline; + private final SparkRuntimeContext runtime; + private final Map rdds = Maps.newHashMap(); + private final Set multireads = Sets.newHashSet(); + private final Map pobjects = Maps.newHashMap(); + + public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { + this.jsc = jsc; + this.pipeline = pipeline; + this.runtime = new SparkRuntimeContext(jsc, pipeline); + } + + JavaSparkContext getSparkContext() { + return jsc; + } + + Pipeline getPipeline() { + return pipeline; + } + + SparkRuntimeContext getRuntimeContext() { + return runtime; + } + + I getInput(PTransform transform) { + return (I) pipeline.getInput(transform); + } + + O getOutput(PTransform transform) { + return (O) pipeline.getOutput(transform); + } + + void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + rdds.put((PValue) getOutput(transform), rdd); + } + + void setPObjectValue(PObject pobject, Object value) { + pobjects.put(pobject, value); + } + + JavaRDDLike getRDD(PValue pvalue) { + JavaRDDLike rdd = rdds.get(pvalue); + if (multireads.contains(pvalue)) { + // Ensure the RDD is marked as cached + rdd.rdd().cache(); + } else { + multireads.add(pvalue); + } + return rdd; + } + + void setRDD(PValue pvalue, JavaRDDLike rdd) { + rdds.put(pvalue, rdd); + } + + JavaRDDLike getInputRDD(PTransform transform) { + return getRDD((PValue) pipeline.getInput(transform)); + } + + BroadcastHelper getBroadcastHelper(PObject value) { + Coder coder = value.getCoder(); + Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(get(value), coder)); + return new BroadcastHelper<>(bcast, coder); + } + + @Override + public T get(PObject value) { + if (pobjects.containsKey(value)) { + return (T) pobjects.get(value); + } else if (rdds.containsKey(value)) { + JavaRDDLike rdd = rdds.get(value); + //TODO: need same logic from get() method below here for serialization of bytes + 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 getAggregatorValue(String named, Class resultType) { + return runtime.getAggregatorValue(named, resultType); + } + + @Override + public Iterable get(PCollection pcollection) { + JavaRDDLike rdd = getRDD(pcollection); + final Coder coder = pcollection.getCoder(); + JavaRDDLike bytes = rdd.map(CoderHelpers.toByteFunction(coder)); + List clientBytes = bytes.collect(); + return Iterables.transform(clientBytes, new Function() { + public T apply(byte[] bytes) { + return (T) CoderHelpers.fromByteArray(bytes, coder); + } + }); + } + + PObjectValueTuple getPObjectTuple(PTransform transform) { + PObjectTuple pot = (PObjectTuple) pipeline.getInput(transform); + PObjectValueTuple povt = PObjectValueTuple.empty(); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + povt = povt.and((TupleTag) e.getKey(), get(e.getValue())); + } + return povt; + } + + void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { + PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); + for (Map.Entry, PObject> e : pot.getAll().entrySet()) { + pobjects.put(e.getValue(), outputValues.get(e.getKey())); + } + } } 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 index 84dec7780aab..0c223e74eacc 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -28,27 +28,27 @@ public interface EvaluationResult extends PipelineResult { * Retrieves an iterable of results associated with the PCollection passed in. * * @param pcollection Collection we wish to translate. - * @param Type of elements contained in collection. + * @param Type of elements contained in collection. * @return Natively types result associated with collection. */ - Iterable get(PCollection pcollection); + Iterable get(PCollection pcollection); /** * Retrieve an object of Type T associated with the PObject passed in. * * @param pobject PObject we wish to translate. - * @param Type of object to return. + * @param Type of object to return. * @return Native object. */ - T get(PObject pobject); + T get(PObject pobject); /** * Retrieves the final value of the aggregator. * - * @param aggName name of aggragtor. + * @param aggName name of aggragtor. * @param resultType Class of final result of aggregatiohn. - * @param Type of final result of aggregation. + * @param Type of final result of aggregation. * @return Result of aggregation associated with specified name. */ - T getAggregatorValue(String aggName, Class resultType); + T getAggregatorValue(String aggName, Class resultType); } 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 index 8200396ddf64..362119666eb2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -14,6 +14,10 @@ */ package com.cloudera.dataflow.spark; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; + import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -31,10 +35,6 @@ import org.joda.time.Instant; import scala.Tuple2; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; - /** * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enrishing the * undelrying data with multiple TupleTags. @@ -43,109 +43,109 @@ * @param Output type for DoFunction. */ class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { - // TODO: I think implementing decoding logic will allow us to do away with having two types of - // DoFunctions. Josh originally made these two classes in order to help ease the typing of - // results. Correctly using coders should just fix this. - - private final DoFn fn; - private final SparkRuntimeContext runtimeContext; - private final TupleTag mainOutputTag; - private final Map, BroadcastHelper> sideInputs; - - public MultiDoFnFunction( - DoFn fn, - SparkRuntimeContext runtimeContext, - TupleTag mainOutputTag, - Map, BroadcastHelper> sideInputs) { - this.fn = fn; - this.runtimeContext = runtimeContext; - this.mainOutputTag = mainOutputTag; - this.sideInputs = sideInputs; - } - - @Override - public Iterable, Object>> call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(fn); - fn.startBundle(ctxt); - while (iter.hasNext()) { - ctxt.element = iter.next(); - fn.processElement(ctxt); - } - fn.finishBundle(ctxt); - return Iterables.transform(ctxt.outputs.entries(), - new Function, Object>, Tuple2, Object>>() { - public Tuple2, Object> apply(Map.Entry, Object> input) { - return new Tuple2, Object>(input.getKey(), input.getValue()); - } - }); - } - - private class ProcCtxt extends DoFn.ProcessContext { - - private Multimap, Object> outputs = LinkedListMultimap.create(); - private I element; - - public ProcCtxt(DoFn fn) { - fn.super(); + // TODO: I think implementing decoding logic will allow us to do away with having two types of + // DoFunctions. Josh originally made these two classes in order to help ease the typing of + // results. Correctly using coders should just fix this. + + private final DoFn mFunction; + private final SparkRuntimeContext mRuntimeContext; + private final TupleTag mMainOutputTag; + private final Map, BroadcastHelper> mSideInputs; + + public MultiDoFnFunction( + DoFn fn, + SparkRuntimeContext runtimeContext, + TupleTag mainOutputTag, + Map, BroadcastHelper> sideInputs) { + this.mFunction = fn; + this.mRuntimeContext = runtimeContext; + this.mMainOutputTag = mainOutputTag; + this. mSideInputs = sideInputs; } @Override - public PipelineOptions getPipelineOptions() { - return runtimeContext.getPipelineOptions(); + public Iterable, Object>> call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt(mFunction); + mFunction.startBundle(ctxt); + while (iter.hasNext()) { + ctxt.element = iter.next(); + mFunction.processElement(ctxt); + } + mFunction.finishBundle(ctxt); + return Iterables.transform(ctxt.outputs.entries(), + new Function, Object>, Tuple2, Object>>() { + public Tuple2, Object> apply(Map.Entry, Object> input) { + return new Tuple2, Object>(input.getKey(), input.getValue()); + } + }); } - @Override - public T sideInput(PCollectionView view) { - return (T) sideInputs.get(view.getTagInternal()).getValue(); - } - - @Override - public synchronized void output(O o) { - outputs.put(mainOutputTag, o); - } - - @Override - public synchronized void sideOutput(TupleTag tag, T t) { - outputs.put(tag, t); - } - - @Override - public Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - return runtimeContext.createAggregator(named, combineFn); - } - - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return runtimeContext.createAggregator(named, sfunc); - } - - @Override - public I element() { - return element; - } - - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public Collection windows() { - return ImmutableList.of(); + private class ProcCtxt extends DoFn.ProcessContext { + + private Multimap, Object> outputs = LinkedListMultimap.create(); + private I element; + + public ProcCtxt(DoFn fn) { + fn.super(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return mRuntimeContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return (T) mSideInputs.get(view.getTagInternal()).getValue(); + } + + @Override + public synchronized void output(O o) { + outputs.put(mMainOutputTag, o); + } + + @Override + public synchronized void sideOutput(TupleTag tag, T t) { + outputs.put(tag, t); + } + + @Override + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return mRuntimeContext.createAggregator(named, combineFn); + } + + @Override + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return mRuntimeContext.createAggregator(named, sfunc); + } + + @Override + public I element() { + return element; + } + + @Override + public DoFn.KeyedState keyedState() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + output(output); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Collection windows() { + return ImmutableList.of(); + } } - } } 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 index 6536804e876a..e8c3fc38ee55 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -15,6 +15,10 @@ package com.cloudera.dataflow.spark; +import java.lang.reflect.Field; +import java.util.Map; +import java.util.logging.Logger; + import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -53,341 +57,327 @@ import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; -import java.lang.reflect.Field; -import java.util.Map; -import java.util.logging.Logger; - /** * 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 in Spark's local mode with two threads, we would do the following: - * Pipeline p = - * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); + * Pipeline p = + * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); */ public class SparkPipelineRunner extends PipelineRunner { - private static final Logger LOG = - Logger.getLogger(SparkPipelineRunner.class.getName()); - /** The url of the spark master to connect to. */ - private final String master; + private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); + private static final Map, TransformEvaluator> mEvaluators = + Maps.newHashMap(); - /** - * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single - * thread. - */ - public SparkPipelineRunner() { - this("local"); - } + + private static TransformEvaluator READ_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + private static TransformEvaluator READ_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + private static TransformEvaluator CREATE = new TransformEvaluator() { + @Override + public void evaluate(Create transform, EvaluationContext context) { + Iterable elems = transform.getElements(); + Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); + JavaRDD rdd = context.getSparkContext().parallelize( + CoderHelpers.toByteArrays(elems, coder)); + context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { + @Override + public void evaluate(CreatePObject transform, EvaluationContext context) { + context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); + } + }; + private static TransformEvaluator TO_ITER = new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterable transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, context.get(in)); + } + }; + /** + * needs to handle coders + */ + private static TransformEvaluator TO_ITER_WIN = + new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, Iterables.transform(context.get(in), + new com.google.common.base.Function() { + @Override + public WindowedValue apply(Object o) { + return WindowedValue.valueInGlobalWindow(o); + } + })); + } + }; + private static TransformEvaluator PARDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + DoFnFunction dofn = new DoFnFunction(transform.getFn(), + context.getRuntimeContext(), + getSideInputs(transform.getSideInputs(), context)); + context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); + } + }; + private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); + private static TransformEvaluator MULTIDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + MultiDoFnFunction multifn = new MultiDoFnFunction( + transform.getFn(), + context.getRuntimeContext(), + (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), + getSideInputs(transform.getSideInputs(), context)); + + JavaPairRDD all = context.getInputRDD(transform) + .mapPartitionsToPair(multifn) + .cache(); + + PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); + for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { + TupleTagFilter filter = new TupleTagFilter(e.getKey()); + JavaPairRDD filtered = all.filter(filter); + context.setRDD(e.getValue(), filtered.values()); + } + } + }; + private static TransformEvaluator SEQDO = new TransformEvaluator() { + @Override + public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { + PObjectValueTuple inputValues = context.getPObjectTuple(transform); + PObjectValueTuple outputValues = transform.getFn().process(inputValues); + context.setPObjectTuple(transform, outputValues); + } + }; + private static TransformEvaluator GBK = new TransformEvaluator() { + @Override + public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { + context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)) + .groupByKey())); + } + }; + private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); + private static TransformEvaluator GROUPED = new TransformEvaluator() { + @Override + public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { + final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { + @Override + public Object call(Object input) throws Exception { + KV kv = (KV) input; + return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); + } + })); + } + }; + private static TransformEvaluator FLATTEN = new TransformEvaluator() { + @Override + public void evaluate(Flatten transform, EvaluationContext context) { + PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + JavaRDD[] rdds = new JavaRDD[pcs.size()]; + for (int i = 0; i < rdds.length; i++) { + rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + } + JavaRDD rdd = context.getSparkContext().union(rdds); + context.setOutputRDD(transform, rdd); + } + }; + /** + * The url of the spark master to connect to. + */ + private final String master; + + /** + * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single + * thread. + */ + public SparkPipelineRunner() { + this("local"); + } /** * Constructor for a pipeline runner. * * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). */ - public SparkPipelineRunner(String master) { - this.master = Preconditions.checkNotNull(master); - } - - @Override - public EvaluationResult run(Pipeline pipeline) { - // TODO: get master from options - JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); - pipeline.traverseTopologically(new Evaluator(ctxt)); - return ctxt; - } - - private JavaSparkContext getContextFromOptions(PipelineOptions options) { - return new JavaSparkContext(master, options.getJobName()); - } - - private static class Evaluator implements Pipeline.PipelineVisitor { - - private final EvaluationContext ctxt; + public SparkPipelineRunner(String master) { + this.master = Preconditions.checkNotNull(master); + } - private Evaluator(EvaluationContext ctxt) { - this.ctxt = ctxt; + private static Map, BroadcastHelper> getSideInputs( + Iterable> views, + EvaluationContext context) { + if (views == null) { + return ImmutableMap.of(); + } else { + Map, BroadcastHelper> sideInputs = Maps.newHashMap(); + for (PCollectionView view : views) { + sideInputs.put(view.getTagInternal(), + context.getBroadcastHelper(view.getPObjectInternal())); + } + return sideInputs; + } } - @Override - public void enterCompositeTransform(TransformTreeNode node) { + private static JavaPairRDD toPair(JavaRDDLike rdd) { + return rdd.mapToPair(new PairFunction() { + @Override + public Tuple2 call(Object o) throws Exception { + KV kv = (KV) o; + return new Tuple2(kv.getKey(), kv.getValue()); + } + }); } - @Override - public void leaveCompositeTransform(TransformTreeNode node) { + private static JavaRDDLike fromPair(JavaPairRDD rdd) { + return rdd.map(new Function() { + @Override + public Object call(Object o) throws Exception { + Tuple2 t2 = (Tuple2) o; + return KV.of(t2._1(), t2._2()); + } + }); } - @Override - public void visitTransform(TransformTreeNode node) { - - PTransform transform = node.getTransform(); - TransformEvaluator evaluator = EVALUATORS.get(transform.getClass()); - if (evaluator == null) { - throw new IllegalStateException( - "no evaluator registered for " + transform); - } - LOG.info("Evaluating " + transform); - evaluator.evaluate(transform, ctxt); + public static void registerEvaluator( + Class transformClass, + TransformEvaluator evaluator) { + mEvaluators.put(transformClass, evaluator); } @Override - public void visitValue(PValue pvalue, TransformTreeNode node) { - } - } - - private static class FieldGetter { - private Map fields; - - public FieldGetter(Class clazz) { - this.fields = Maps.newHashMap(); - for (Field f : clazz.getDeclaredFields()) { - try { - f.setAccessible(true); - this.fields.put(f.getName(), f); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + public EvaluationResult run(Pipeline pipeline) { + // TODO: get master from options + JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); + pipeline.traverseTopologically(new Evaluator(ctxt)); + return ctxt; } - public T get(String fieldname, Object value) { - try { - return (T) fields.get(fieldname).get(value); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } + private JavaSparkContext getContextFromOptions(PipelineOptions options) { + return new JavaSparkContext(master, options.getJobName()); } - } - private static TransformEvaluator READ_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; + private static class Evaluator implements Pipeline.PipelineVisitor { - private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; + private final EvaluationContext ctxt; - private static TransformEvaluator READ_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; + private Evaluator(EvaluationContext ctxt) { + this.ctxt = ctxt; + } - private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; + @Override + public void enterCompositeTransform(TransformTreeNode node) { + } - private static TransformEvaluator CREATE = new TransformEvaluator() { - @Override - public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = transform.getElements(); - Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); - JavaRDD rdd = context.getSparkContext().parallelize( - CoderHelpers.toByteArrays(elems, coder)); - context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); - } - }; + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } - private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { - @Override - public void evaluate(CreatePObject transform, EvaluationContext context) { - context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); - } - }; + @Override + public void visitTransform(TransformTreeNode node) { - private static TransformEvaluator TO_ITER = new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterable transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, context.get(in)); - } - }; - /** - * needs to handle coders - */ - private static TransformEvaluator TO_ITER_WIN = - new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, Iterables.transform(context.get(in), - new com.google.common.base.Function() { - @Override - public WindowedValue apply(Object o) { - return WindowedValue.valueInGlobalWindow(o); + PTransform transform = node.getTransform(); + TransformEvaluator evaluator = mEvaluators.get(transform.getClass()); + if (evaluator == null) { + throw new IllegalStateException( + "no evaluator registered for " + transform); } - })); - } - }; - - private static Map, BroadcastHelper> getSideInputs( - Iterable> views, - EvaluationContext context) { - if (views == null) { - return ImmutableMap.of(); - } else { - Map, BroadcastHelper>sideInputs = Maps.newHashMap(); - for (PCollectionView view : views) { - sideInputs.put(view.getTagInternal(), context.getBroadcastHelper(view.getPObjectInternal())); - } - return sideInputs; - } - } - - private static TransformEvaluator PARDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.Bound transform, EvaluationContext context) { - DoFnFunction dofn = new DoFnFunction(transform.getFn(), - context.getRuntimeContext(), - getSideInputs(transform.getSideInputs(), context)); - context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); - } - }; + LOG.info("Evaluating " + transform); + evaluator.evaluate(transform, ctxt); + } - private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); - private static TransformEvaluator MULTIDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - MultiDoFnFunction multifn = new MultiDoFnFunction( - transform.getFn(), - context.getRuntimeContext(), - (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), - getSideInputs(transform.getSideInputs(), context)); - - JavaPairRDD all = context.getInputRDD(transform) - .mapPartitionsToPair(multifn) - .cache(); - - PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); - for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { - TupleTagFilter filter = new TupleTagFilter(e.getKey()); - JavaPairRDD filtered = all.filter(filter); - context.setRDD(e.getValue(), filtered.values()); - } + @Override + public void visitValue(PValue pvalue, TransformTreeNode node) { + } } - }; - private static class TupleTagFilter implements Function, Boolean> { - private TupleTag tag; + private static class FieldGetter { + private Map fields; + + public FieldGetter(Class clazz) { + this.fields = Maps.newHashMap(); + for (Field f : clazz.getDeclaredFields()) { + try { + f.setAccessible(true); + this.fields.put(f.getName(), f); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } - public TupleTagFilter(TupleTag tag) { - this.tag = tag; + public T get(String fieldname, Object value) { + try { + return (T) fields.get(fieldname).get(value); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } } - @Override - public Boolean call(Tuple2 input) throws Exception { - return tag.equals(input._1()); - } - } + private static class TupleTagFilter implements Function, Boolean> { + private TupleTag tag; - private static TransformEvaluator SEQDO = new TransformEvaluator() { - @Override - public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { - PObjectValueTuple inputValues = context.getPObjectTuple(transform); - PObjectValueTuple outputValues = transform.getFn().process(inputValues); - context.setPObjectTuple(transform, outputValues); - } - }; - - private static JavaPairRDD toPair(JavaRDDLike rdd) { - return rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Object o) throws Exception { - KV kv = (KV) o; - return new Tuple2(kv.getKey(), kv.getValue()); - } - }); - } - - private static JavaRDDLike fromPair(JavaPairRDD rdd) { - return rdd.map(new Function() { - @Override - public Object call(Object o) throws Exception { - Tuple2 t2 = (Tuple2) o; - return KV.of(t2._1(), t2._2()); - } - }); - } - - private static TransformEvaluator GBK = new TransformEvaluator() { - @Override - public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { - context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); - } - }; + public TupleTagFilter(TupleTag tag) { + this.tag = tag; + } - private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); - private static TransformEvaluator GROUPED = new TransformEvaluator() { - @Override - public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); - context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { @Override - public Object call(Object input) throws Exception { - KV kv = (KV) input; - return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); + public Boolean call(Tuple2 input) throws Exception { + return tag.equals(input._1()); } - })); } - }; - private static TransformEvaluator FLATTEN = new TransformEvaluator() { - @Override - public void evaluate(Flatten transform, EvaluationContext context) { - PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); - JavaRDD[] rdds = new JavaRDD[pcs.size()]; - for (int i = 0; i < rdds.length; i++) { - rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); - } - JavaRDD rdd = context.getSparkContext().union(rdds); - context.setOutputRDD(transform, rdd); + static { + registerEvaluator(TextIO.Read.Bound.class, READ_TEXT); + registerEvaluator(TextIO.Write.Bound.class, WRITE_TEXT); + registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); + registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); + registerEvaluator(ParDo.Bound.class, PARDO); + registerEvaluator(ParDo.BoundMulti.class, MULTIDO); + registerEvaluator(SeqDo.BoundMulti.class, SEQDO); + registerEvaluator(GroupByKey.GroupByKeyOnly.class, GBK); + registerEvaluator(Combine.GroupedValues.class, GROUPED); + registerEvaluator(Flatten.class, FLATTEN); + registerEvaluator(Create.class, CREATE); + registerEvaluator(CreatePObject.class, CREATE_POBJ); + registerEvaluator(Convert.ToIterable.class, TO_ITER); + registerEvaluator(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); } - }; - - public static void registerEvaluator( - Class transformClass, - TransformEvaluator evaluator) { - EVALUATORS.put(transformClass, evaluator); - } - - /** - * helps map from the functions being applied to transform evaluations - */ - private static final Map, TransformEvaluator> EVALUATORS = Maps.newHashMap(); - static { - registerEvaluator(TextIO.Read.Bound.class, READ_TEXT); - registerEvaluator(TextIO.Write.Bound.class, WRITE_TEXT); - registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); - registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); - registerEvaluator(ParDo.Bound.class, PARDO); - registerEvaluator(ParDo.BoundMulti.class, MULTIDO); - registerEvaluator(SeqDo.BoundMulti.class, SEQDO); - registerEvaluator(GroupByKey.GroupByKeyOnly.class, GBK); - registerEvaluator(Combine.GroupedValues.class, GROUPED); - registerEvaluator(Flatten.class, FLATTEN); - registerEvaluator(Create.class, CREATE); - registerEvaluator(CreatePObject.class, CREATE_POBJ); - registerEvaluator(Convert.ToIterable.class, TO_ITER); - registerEvaluator(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); - } } 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 index 2af94b18a7a2..7c1b46745287 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -14,8 +14,10 @@ */ package com.cloudera.dataflow.spark; -import com.cloudera.dataflow.spark.aggregators.AggAccumParam; -import com.cloudera.dataflow.spark.aggregators.NamedAggregators; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; @@ -24,101 +26,105 @@ import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; +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. */ class SparkRuntimeContext implements Serializable { - /** An accumulator that is a map from names to aggregators. */ - private Accumulator accum; - /** Map fo names to dataflow aggregators. */ - private Map aggregators = new HashMap<>(); + /** + * An accumulator that is a map from names to aggregators. + */ + private Accumulator accum; + /** + * Map fo names to dataflow aggregators. + */ + private Map aggregators = new HashMap<>(); - public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); - } + public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { + this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); + } - /** - * 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 Type of object to be returned. - * @return The value of the aggregator. - */ - public T getAggregatorValue(String aggregatorName, Class typeClass) { - return accum.value().getValue(aggregatorName, typeClass); - } + /** + * 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 Type of object to be returned. + * @return The value of the aggregator. + */ + public T getAggregatorValue(String aggregatorName, Class typeClass) { + return accum.value().getValue(aggregatorName, typeClass); + } - public synchronized PipelineOptions getPipelineOptions() { - //TODO: Support this. - throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); - } + public synchronized PipelineOptions getPipelineOptions() { + //TODO: Support this. + throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); + } - /** - * Creates and aggregator and associates it with the specified name. - * - * @param named Name of aggregator. - * @param sfunc Serializable function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. - * @return Specified aggregator - */ - public synchronized Aggregator createAggregator( - String named, - SerializableFunction, Out> sfunc) { - Aggregator aggregator = aggregators.get(named); - if (aggregator == null) { - NamedAggregators.SerFunctionState state = new NamedAggregators - .SerFunctionState<>(sfunc); - accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator(state); - aggregators.put(named, aggregator); + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param sfunc Serializable function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( + String named, + SerializableFunction, Out> sfunc) { + Aggregator aggregator = aggregators.get(named); + if (aggregator == null) { + NamedAggregators.SerFunctionState state = new NamedAggregators + .SerFunctionState<>(sfunc); + accum.add(new NamedAggregators(named, state)); + aggregator = new SparkAggregator(state); + aggregators.put(named, aggregator); + } + return aggregator; } - return aggregator; - } - /** - * Creates and aggregator and associates it with the specified name. - * - * @param named Name of aggregator. - * @param combineFn Combine function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. - * @return Specified aggregator - */ - public synchronized Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - Aggregator aggregator = aggregators.get(named); - if (aggregator == null) { - NamedAggregators.CombineFunctionState state = new NamedAggregators - .CombineFunctionState<>(combineFn); - accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator(state); - aggregators.put(named, aggregator); + + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param combineFn Combine function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + Aggregator aggregator = aggregators.get(named); + if (aggregator == null) { + NamedAggregators.CombineFunctionState state = new NamedAggregators + .CombineFunctionState<>(combineFn); + accum.add(new NamedAggregators(named, state)); + aggregator = new SparkAggregator(state); + aggregators.put(named, aggregator); + } + return aggregator; } - return aggregator; - } - /** - * Initialize spark aggregators exactly once. - * - * @param Type of element fed in to aggregator. - */ - private static class SparkAggregator implements Aggregator { - private final NamedAggregators.State state; + /** + * Initialize spark aggregators exactly once. + * + * @param Type of element fed in to aggregator. + */ + private static class SparkAggregator implements Aggregator { + private final NamedAggregators.State state; - public SparkAggregator(NamedAggregators.State state) { - this.state = state; - } + public SparkAggregator(NamedAggregators.State state) { + this.state = state; + } - @Override - public void addValue(In elem) { - state.update(elem); + @Override + public void addValue(In elem) { + state.update(elem); + } } - } } 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 index f86533203389..6173920681c7 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java @@ -15,10 +15,10 @@ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.transforms.PTransform; - import java.io.Serializable; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + public interface TransformEvaluator extends Serializable { - void evaluate(PT transform, EvaluationContext context); + void evaluate(PT transform, EvaluationContext context); } 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 index a3055a256ded..918c52c98b2e 100644 --- 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 @@ -18,18 +18,18 @@ import org.apache.spark.AccumulatorParam; public class AggAccumParam implements AccumulatorParam { - @Override - public NamedAggregators addAccumulator(NamedAggregators current, NamedAggregators added) { - return current.merge(added); - } + @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 addInPlace(NamedAggregators current, NamedAggregators added) { + return addAccumulator(current, added); + } - @Override - public NamedAggregators zero(NamedAggregators initialValue) { - return new NamedAggregators(); - } + @Override + public NamedAggregators zero(NamedAggregators initialValue) { + return new NamedAggregators(); + } } 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 index 61cd9278a9fa..7e45c95da917 100644 --- 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 @@ -15,14 +15,14 @@ package com.cloudera.dataflow.spark.aggregators; -import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.common.collect.ImmutableList; - import java.io.Serializable; import java.util.Map; import java.util.TreeMap; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.common.collect.ImmutableList; + /** * What is an Agg? * wrapper around a map of named aggregators. @@ -31,146 +31,149 @@ * When the dataflow */ public class NamedAggregators implements Serializable { - /** - * - * @param Input data type - * @param Intermediate data type (useful for averages) - * @param Output datatype - */ - public interface State extends Serializable { - /** - * - * @param element - */ - void update(In element); - State merge(State other); - Inter current(); - Out render(); - } - /** * Why is this final if you later add states to it? */ - private final Map mNamedAggregators = new TreeMap<>(); + private final Map mNamedAggregators = new TreeMap<>(); - public NamedAggregators() { - } + public NamedAggregators() { + } /** * is "named" the label for a state? + * * @param named * @param state */ - public NamedAggregators(String named, State state) { - this.mNamedAggregators.put(named, state); - } + public NamedAggregators(String named, State state) { + this.mNamedAggregators.put(named, state); + } /** - * * @param named * @param typeClass * @param * @return */ - public T getValue(String named, Class typeClass) { - return typeClass.cast(mNamedAggregators.get(named).render()); - } - - public NamedAggregators merge(NamedAggregators other) { - for (Map.Entry e : other.mNamedAggregators.entrySet()) { - State cur = mNamedAggregators.get(e.getKey()); - if (cur == null) { - mNamedAggregators.put(e.getKey(), e.getValue()); - } else { - mNamedAggregators.put(e.getKey(), cur.merge(e.getValue())); - } + public T getValue(String named, Class typeClass) { + return typeClass.cast(mNamedAggregators.get(named).render()); } - return this; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Map.Entry e : mNamedAggregators.entrySet()) { - sb.append(e.getKey()).append(": ").append(e.getValue().render()); - } - return sb.toString(); - } - /** - * => combineFunction in data flow. - * @param - * @param - * @param - */ - public static class CombineFunctionState implements State { - - private Combine.CombineFn combineFn; - private Inter state; - - public CombineFunctionState(Combine.CombineFn combineFn) { - this.combineFn = combineFn; - this.state = combineFn.createAccumulator(); + public NamedAggregators merge(NamedAggregators other) { + for (Map.Entry e : other.mNamedAggregators.entrySet()) { + State cur = mNamedAggregators.get(e.getKey()); + if (cur == null) { + mNamedAggregators.put(e.getKey(), e.getValue()); + } else { + mNamedAggregators.put(e.getKey(), cur.merge(e.getValue())); + } + } + return this; } @Override - public void update(In element) { - combineFn.addInput(state, element); + public String toString() { + StringBuilder sb = new StringBuilder(); + for (Map.Entry e : mNamedAggregators.entrySet()) { + sb.append(e.getKey()).append(": ").append(e.getValue().render()); + } + return sb.toString(); } - @Override - public State merge(State other) { - this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); - return this; - } + /** + * @param Input data type + * @param Intermediate data type (useful for averages) + * @param Output datatype + */ + public interface State extends Serializable { + /** + * @param element + */ + void update(In element); - @Override - public Inter current() { - return state; - } + State merge(State other); - @Override - public Out render() { - return combineFn.extractOutput(state); + Inter current(); + + Out render(); } - } /** - * states correspond to dataflow objects. this one => seriazable function + * => combineFunction in data flow. + * * @param + * @param * @param */ - public static class SerFunctionState implements State { - - private SerializableFunction, Out> sfunc; - private Out state; - - public SerFunctionState(SerializableFunction, Out> sfunc) { - this.sfunc = sfunc; - this.state = sfunc.apply(ImmutableList.of()); - } - - @Override - public void update(In element) { - this.state = sfunc.apply(ImmutableList.of(element, (In) state)); - } - - @Override - public State merge(State other) { - // Add exception catching and logging here. - this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); - return this; - } - - @Override - public Out current() { - return state; + public static class CombineFunctionState implements State { + + private Combine.CombineFn combineFn; + private Inter state; + + public CombineFunctionState(Combine.CombineFn combineFn) { + this.combineFn = combineFn; + this.state = combineFn.createAccumulator(); + } + + @Override + public void update(In element) { + combineFn.addInput(state, element); + } + + @Override + public State merge(State 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 Out render() { - return state; + /** + * states correspond to dataflow objects. this one => seriazable function + * + * @param + * @param + */ + public static class SerFunctionState implements State { + + private SerializableFunction, Out> sfunc; + private Out state; + + public SerFunctionState(SerializableFunction, Out> sfunc) { + this.sfunc = sfunc; + this.state = sfunc.apply(ImmutableList.of()); + } + + @Override + public void update(In element) { + this.state = sfunc.apply(ImmutableList.of(element, (In) state)); + } + + @Override + public State merge(State other) { + // Add exception catching and logging here. + this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); + return this; + } + + @Override + public Out current() { + return state; + } + + @Override + public Out render() { + return state; + } } - } } diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 81cd36b9e5e8..6c73fa67102b 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -41,92 +41,95 @@ public class WordCountTest { - /** A DoFn that tokenizes lines of text into individual words. */ - static class ExtractWordsFn extends DoFn { + static TupleTag upper = new TupleTag<>(); + static TupleTag lower = new TupleTag<>(); + static TupleTag> lowerCnts = new TupleTag<>(); + static TupleTag> upperCnts = new TupleTag<>(); - Aggregator totalWords; - Aggregator maxWordLength; - PCollectionView regex; + @Test + public void testRun() throws Exception { + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection regex = p.apply(Create.of("[^a-zA-Z']+")); + PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); + PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); + PCollectionList list = PCollectionList.of(w1).and(w2); - public ExtractWordsFn(PCollectionView regex) { - this.regex = regex; - } - @Override - public void startBundle(Context ctxt) { - this.totalWords = ctxt.createAggregator("totalWords", - new Sum.SumIntegerFn()); - this.maxWordLength = ctxt.createAggregator("maxWordLength", - new Max.MaxIntegerFn()); - } + PCollection union = list.apply(Flatten.create()); + PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); + PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); - @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); - } - } - } + EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + Iterable> actualLower = res.get(luc.get(lowerCnts)); + Iterable> actualUpper = res.get(luc.get(upperCnts)); + Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); + Iterable 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); } - } - static TupleTag upper = new TupleTag<>(); - static TupleTag lower = new TupleTag<>(); - static TupleTag> lowerCnts = new TupleTag<>(); - static TupleTag> upperCnts = new TupleTag<>(); + /** + * A DoFn that tokenizes lines of text into individual words. + */ + static class ExtractWordsFn extends DoFn { - public static class CountWords extends PTransform, PCollectionTuple> { + Aggregator totalWords; + Aggregator maxWordLength; + PCollectionView regex; - private final PCollectionView regex; + public ExtractWordsFn(PCollectionView regex) { + this.regex = regex; + } + + @Override + public void startBundle(Context ctxt) { + this.totalWords = ctxt.createAggregator("totalWords", + new Sum.SumIntegerFn()); + this.maxWordLength = ctxt.createAggregator("maxWordLength", + new Max.MaxIntegerFn()); + } - public CountWords(PCollectionView 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); + } + } + } + } } - @Override - public PCollectionTuple apply(PCollection lines) { - // Convert lines of text into individual words. - PCollectionTuple lowerUpper = lines - .apply(ParDo.of(new ExtractWordsFn(regex)) - .withSideInputs(regex) - .withOutputTags(lower, TupleTagList.of(upper))); - lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); - lowerUpper.get(upper).setCoder(StringUtf8Coder.of()); - PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.perElement()); - PCollection> upperCounts = lowerUpper.get(upper).apply(Count.perElement()); - return PCollectionTuple - .of(lowerCnts, lowerCounts) - .and(upperCnts, upperCounts); + public static class CountWords extends PTransform, PCollectionTuple> { + + private final PCollectionView regex; + + public CountWords(PCollectionView regex) { + this.regex = regex; + } + + @Override + public PCollectionTuple apply(PCollection lines) { + // Convert lines of text into individual words. + PCollectionTuple lowerUpper = lines + .apply(ParDo.of(new ExtractWordsFn(regex)) + .withSideInputs(regex) + .withOutputTags(lower, TupleTagList.of(upper))); + lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); + lowerUpper.get(upper).setCoder(StringUtf8Coder.of()); + PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.perElement()); + PCollection> upperCounts = lowerUpper.get(upper).apply(Count.perElement()); + return PCollectionTuple + .of(lowerCnts, lowerCounts) + .and(upperCnts, upperCounts); + } } - } - - @Test - public void testRun() throws Exception { - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - PCollection regex = p.apply(Create.of("[^a-zA-Z']+")); - PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); - PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); - PCollectionList list = PCollectionList.of(w1).and(w2); - - PCollection union = list.apply(Flatten.create()); - PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); - PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); - - EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); - Iterable> actualLower = res.get(luc.get(lowerCnts)); - Iterable> actualUpper = res.get(luc.get(upperCnts)); - Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); - Iterable 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); - } } From 2992838fe35e06f280fbbf2eb419973be7699281 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Thu, 11 Dec 2014 16:56:01 -0800 Subject: [PATCH 022/137] Factor out tranform translation logic in to its own class. --- .../dataflow/spark/SparkPipelineRunner.java | 348 +++--------------- .../dataflow/spark/TransformTranslator.java | 279 ++++++++++++++ 2 files changed, 328 insertions(+), 299 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.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 index e8c3fc38ee55..4986db79e0f5 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -15,10 +15,6 @@ package com.cloudera.dataflow.spark; -import java.lang.reflect.Field; -import java.util.Map; -import java.util.logging.Logger; - import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -57,327 +53,81 @@ import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; +import java.lang.reflect.Field; +import java.util.Map; +import java.util.logging.Logger; + /** * 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 in Spark's local mode with two threads, we would do the following: - * Pipeline p = - * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); + * Pipeline p = + * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); */ public class SparkPipelineRunner extends PipelineRunner { - private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); - private static final Map, TransformEvaluator> mEvaluators = - Maps.newHashMap(); - - - private static TransformEvaluator READ_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; - private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; - private static TransformEvaluator READ_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; - private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; - private static TransformEvaluator CREATE = new TransformEvaluator() { - @Override - public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = transform.getElements(); - Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); - JavaRDD rdd = context.getSparkContext().parallelize( - CoderHelpers.toByteArrays(elems, coder)); - context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); - } - }; - private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { - @Override - public void evaluate(CreatePObject transform, EvaluationContext context) { - context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); - } - }; - private static TransformEvaluator TO_ITER = new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterable transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, context.get(in)); - } - }; - /** - * needs to handle coders - */ - private static TransformEvaluator TO_ITER_WIN = - new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, Iterables.transform(context.get(in), - new com.google.common.base.Function() { - @Override - public WindowedValue apply(Object o) { - return WindowedValue.valueInGlobalWindow(o); - } - })); - } - }; - private static TransformEvaluator PARDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.Bound transform, EvaluationContext context) { - DoFnFunction dofn = new DoFnFunction(transform.getFn(), - context.getRuntimeContext(), - getSideInputs(transform.getSideInputs(), context)); - context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); - } - }; - private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); - private static TransformEvaluator MULTIDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - MultiDoFnFunction multifn = new MultiDoFnFunction( - transform.getFn(), - context.getRuntimeContext(), - (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), - getSideInputs(transform.getSideInputs(), context)); - - JavaPairRDD all = context.getInputRDD(transform) - .mapPartitionsToPair(multifn) - .cache(); + private static final Logger LOG = + Logger.getLogger(SparkPipelineRunner.class.getName()); + /** The url of the spark master to connect to. */ + private final String master; - PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); - for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { - TupleTagFilter filter = new TupleTagFilter(e.getKey()); - JavaPairRDD filtered = all.filter(filter); - context.setRDD(e.getValue(), filtered.values()); - } - } - }; - private static TransformEvaluator SEQDO = new TransformEvaluator() { - @Override - public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { - PObjectValueTuple inputValues = context.getPObjectTuple(transform); - PObjectValueTuple outputValues = transform.getFn().process(inputValues); - context.setPObjectTuple(transform, outputValues); - } - }; - private static TransformEvaluator GBK = new TransformEvaluator() { - @Override - public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { - context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)) - .groupByKey())); - } - }; - private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); - private static TransformEvaluator GROUPED = new TransformEvaluator() { - @Override - public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); - context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { - @Override - public Object call(Object input) throws Exception { - KV kv = (KV) input; - return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); - } - })); - } - }; - private static TransformEvaluator FLATTEN = new TransformEvaluator() { - @Override - public void evaluate(Flatten transform, EvaluationContext context) { - PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); - JavaRDD[] rdds = new JavaRDD[pcs.size()]; - for (int i = 0; i < rdds.length; i++) { - rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); - } - JavaRDD rdd = context.getSparkContext().union(rdds); - context.setOutputRDD(transform, rdd); - } - }; - /** - * The url of the spark master to connect to. - */ - private final String master; - - /** - * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single - * thread. - */ - public SparkPipelineRunner() { - this("local"); - } + /** + * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single + * thread. + */ + public SparkPipelineRunner() { + this("local"); + } /** * Constructor for a pipeline runner. * * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). */ - public SparkPipelineRunner(String master) { - this.master = Preconditions.checkNotNull(master); - } + public SparkPipelineRunner(String master) { + this.master = Preconditions.checkNotNull(master); + } - private static Map, BroadcastHelper> getSideInputs( - Iterable> views, - EvaluationContext context) { - if (views == null) { - return ImmutableMap.of(); - } else { - Map, BroadcastHelper> sideInputs = Maps.newHashMap(); - for (PCollectionView view : views) { - sideInputs.put(view.getTagInternal(), - context.getBroadcastHelper(view.getPObjectInternal())); - } - return sideInputs; - } - } + @Override + public EvaluationResult run(Pipeline pipeline) { + // TODO: get master from options + JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); + pipeline.traverseTopologically(new Evaluator(ctxt)); + return ctxt; + } - private static JavaPairRDD toPair(JavaRDDLike rdd) { - return rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Object o) throws Exception { - KV kv = (KV) o; - return new Tuple2(kv.getKey(), kv.getValue()); - } - }); - } + private JavaSparkContext getContextFromOptions(PipelineOptions options) { + return new JavaSparkContext(master, options.getJobName()); + } - private static JavaRDDLike fromPair(JavaPairRDD rdd) { - return rdd.map(new Function() { - @Override - public Object call(Object o) throws Exception { - Tuple2 t2 = (Tuple2) o; - return KV.of(t2._1(), t2._2()); - } - }); - } - - public static void registerEvaluator( - Class transformClass, - TransformEvaluator evaluator) { - mEvaluators.put(transformClass, evaluator); - } + private static class Evaluator implements Pipeline.PipelineVisitor { - @Override - public EvaluationResult run(Pipeline pipeline) { - // TODO: get master from options - JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); - pipeline.traverseTopologically(new Evaluator(ctxt)); - return ctxt; - } + private final EvaluationContext ctxt; - private JavaSparkContext getContextFromOptions(PipelineOptions options) { - return new JavaSparkContext(master, options.getJobName()); + private Evaluator(EvaluationContext ctxt) { + this.ctxt = ctxt; } - private static class Evaluator implements Pipeline.PipelineVisitor { - - private final EvaluationContext ctxt; - - private Evaluator(EvaluationContext ctxt) { - this.ctxt = ctxt; - } - - @Override - public void enterCompositeTransform(TransformTreeNode node) { - } - - @Override - public void leaveCompositeTransform(TransformTreeNode node) { - } - - @Override - public void visitTransform(TransformTreeNode node) { - - PTransform transform = node.getTransform(); - TransformEvaluator evaluator = mEvaluators.get(transform.getClass()); - if (evaluator == null) { - throw new IllegalStateException( - "no evaluator registered for " + transform); - } - LOG.info("Evaluating " + transform); - evaluator.evaluate(transform, ctxt); - } - - @Override - public void visitValue(PValue pvalue, TransformTreeNode node) { - } + @Override + public void enterCompositeTransform(TransformTreeNode node) { } - private static class FieldGetter { - private Map fields; - - public FieldGetter(Class clazz) { - this.fields = Maps.newHashMap(); - for (Field f : clazz.getDeclaredFields()) { - try { - f.setAccessible(true); - this.fields.put(f.getName(), f); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - public T get(String fieldname, Object value) { - try { - return (T) fields.get(fieldname).get(value); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } - } + @Override + public void leaveCompositeTransform(TransformTreeNode node) { } - private static class TupleTagFilter implements Function, Boolean> { - private TupleTag tag; - - public TupleTagFilter(TupleTag tag) { - this.tag = tag; - } - - @Override - public Boolean call(Tuple2 input) throws Exception { - return tag.equals(input._1()); - } + @Override + public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + TransformEvaluator evaluator = TransformTranslator.getTransformEvaluator(transform.getClass()); + LOG.info("Evaluating " + transform); + evaluator.evaluate(transform, ctxt); } - static { - registerEvaluator(TextIO.Read.Bound.class, READ_TEXT); - registerEvaluator(TextIO.Write.Bound.class, WRITE_TEXT); - registerEvaluator(AvroIO.Read.Bound.class, READ_AVRO); - registerEvaluator(AvroIO.Write.Bound.class, WRITE_AVRO); - registerEvaluator(ParDo.Bound.class, PARDO); - registerEvaluator(ParDo.BoundMulti.class, MULTIDO); - registerEvaluator(SeqDo.BoundMulti.class, SEQDO); - registerEvaluator(GroupByKey.GroupByKeyOnly.class, GBK); - registerEvaluator(Combine.GroupedValues.class, GROUPED); - registerEvaluator(Flatten.class, FLATTEN); - registerEvaluator(Create.class, CREATE); - registerEvaluator(CreatePObject.class, CREATE_POBJ); - registerEvaluator(Convert.ToIterable.class, TO_ITER); - registerEvaluator(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); + @Override + public void visitValue(PValue pvalue, TransformTreeNode node) { } + } } + 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 new file mode 100644 index 000000000000..e163de2afd3f --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -0,0 +1,279 @@ +package com.cloudera.dataflow.spark; + +import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.*; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +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.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import scala.Tuple2; + +import java.lang.reflect.Field; +import java.util.Map; + +/** + * Supports translation between a DataFlow transform, and Spark's operations on RDDs. + */ +public class TransformTranslator { + private static class FieldGetter { + private Map fields; + + public FieldGetter(Class clazz) { + this.fields = Maps.newHashMap(); + for (Field f : clazz.getDeclaredFields()) { + try { + f.setAccessible(true); + this.fields.put(f.getName(), f); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + public T get(String fieldname, Object value) { + try { + return (T) fields.get(fieldname).get(value); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + + private static TransformEvaluator FLATTEN = new TransformEvaluator() { + @Override + public void evaluate(Flatten transform, EvaluationContext context) { + PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + JavaRDD[] rdds = new JavaRDD[pcs.size()]; + for (int i = 0; i < rdds.length; i++) { + rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + } + JavaRDD rdd = context.getSparkContext().union(rdds); + context.setOutputRDD(transform, rdd); + } + }; + private static TransformEvaluator GBK = new TransformEvaluator() { + @Override + public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { + context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); + } + }; + + private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); + private static TransformEvaluator GROUPED = new TransformEvaluator() { + @Override + public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { + final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { + @Override + public Object call(Object input) throws Exception { + KV kv = (KV) input; + return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); + } + })); + } + }; + + private static JavaPairRDD toPair(JavaRDDLike rdd) { + return rdd.mapToPair(new PairFunction() { + @Override + public Tuple2 call(Object o) throws Exception { + KV kv = (KV) o; + return new Tuple2(kv.getKey(), kv.getValue()); + } + }); + } + + private static JavaRDDLike fromPair(JavaPairRDD rdd) { + return rdd.map(new Function() { + @Override + public Object call(Object o) throws Exception { + Tuple2 t2 = (Tuple2) o; + return KV.of(t2._1(), t2._2()); + } + }); + } + + + private static TransformEvaluator PARDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + DoFnFunction dofn = new DoFnFunction(transform.getFn(), + context.getRuntimeContext(), + getSideInputs(transform.getSideInputs(), context)); + context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); + } + }; + + private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); + private static TransformEvaluator MULTIDO = new TransformEvaluator() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + MultiDoFnFunction multifn = new MultiDoFnFunction( + transform.getFn(), + context.getRuntimeContext(), + (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), + getSideInputs(transform.getSideInputs(), context)); + + JavaPairRDD all = context.getInputRDD(transform) + .mapPartitionsToPair(multifn) + .cache(); + + PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); + for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { + TupleTagFilter filter = new TupleTagFilter(e.getKey()); + JavaPairRDD filtered = all.filter(filter); + context.setRDD(e.getValue(), filtered.values()); + } + } + }; + + + private static TransformEvaluator READ_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + + private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { + @Override + public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + + private static TransformEvaluator READ_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + + private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { + @Override + public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { + JavaRDDLike last = context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + + private static TransformEvaluator CREATE = new TransformEvaluator() { + @Override + public void evaluate(Create transform, EvaluationContext context) { + Iterable elems = transform.getElements(); + Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); + JavaRDD rdd = context.getSparkContext().parallelize( + CoderHelpers.toByteArrays(elems, coder)); + context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + + private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { + @Override + public void evaluate(CreatePObject transform, EvaluationContext context) { + context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); + } + }; + + private static TransformEvaluator TO_ITER = new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterable transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, context.get(in)); + } + }; + + private static TransformEvaluator TO_ITER_WIN = + new TransformEvaluator() { + @Override + public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { + PCollection in = (PCollection) context.getInput(transform); + PObject out = (PObject) context.getOutput(transform); + context.setPObjectValue(out, Iterables.transform(context.get(in), + new com.google.common.base.Function() { + @Override + public WindowedValue apply(Object o) { + return WindowedValue.valueInGlobalWindow(o); + } + })); + } + }; + + private static class TupleTagFilter implements Function, Boolean> { + private TupleTag tag; + + public TupleTagFilter(TupleTag tag) { + this.tag = tag; + } + + @Override + public Boolean call(Tuple2 input) throws Exception { + return tag.equals(input._1()); + } + } + + private static TransformEvaluator SEQDO = new TransformEvaluator() { + @Override + public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { + PObjectValueTuple inputValues = context.getPObjectTuple(transform); + PObjectValueTuple outputValues = transform.getFn().process(inputValues); + context.setPObjectTuple(transform, outputValues); + } + }; + + private static Map, BroadcastHelper> getSideInputs( + Iterable> views, + EvaluationContext context) { + if (views == null) { + return ImmutableMap.of(); + } else { + Map, BroadcastHelper>sideInputs = Maps.newHashMap(); + for (PCollectionView view : views) { + sideInputs.put(view.getTagInternal(), context.getBroadcastHelper(view.getPObjectInternal())); + } + return sideInputs; + } + } + + private static final Map, TransformEvaluator> mEvaluators = Maps.newHashMap(); + static { + mEvaluators.put(TextIO.Read.Bound.class, READ_TEXT); + mEvaluators.put(TextIO.Write.Bound.class, WRITE_TEXT); + mEvaluators.put(AvroIO.Read.Bound.class, READ_AVRO); + mEvaluators.put(AvroIO.Write.Bound.class, WRITE_AVRO); + mEvaluators.put(ParDo.Bound.class, PARDO); + mEvaluators.put(ParDo.BoundMulti.class, MULTIDO); + mEvaluators.put(SeqDo.BoundMulti.class, SEQDO); + mEvaluators.put(GroupByKey.GroupByKeyOnly.class, GBK); + mEvaluators.put(Combine.GroupedValues.class, GROUPED); + mEvaluators.put(Flatten.class, FLATTEN); + mEvaluators.put(Create.class, CREATE); + mEvaluators.put(CreatePObject.class, CREATE_POBJ); + mEvaluators.put(Convert.ToIterable.class, TO_ITER); + mEvaluators.put(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); + } + + public static TransformEvaluator getTransformEvaluator(Class clazz) { + TransformEvaluator transform = mEvaluators.get(clazz); + if (transform == null) { + throw new IllegalStateException("No TransformEvaluator registered for " + clazz); + } + return transform; + } +} From 2e3fe1ab9412be1b10659cd273a4de0ec5f103cd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 13 Dec 2014 16:03:39 +0000 Subject: [PATCH 023/137] Specify and rationalize generic types in State, CoderHelpers to start --- .../cloudera/dataflow/spark/CoderHelpers.java | 6 +- .../spark/aggregators/NamedAggregators.java | 284 ++++++++++-------- .../dataflow/spark/WordCountTest.java | 165 +++++----- 3 files changed, 238 insertions(+), 217 deletions(-) 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 index 03f02f4adb68..cd15f1a3501d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -34,7 +34,7 @@ public class CoderHelpers { * @param coder Coder to serialize with. * @return Byte array representing serialized object. */ - static byte[] toByteArray(Object value, Coder coder) { + static byte[] toByteArray(T value, Coder coder) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); try { coder.encode(value, baos, new Coder.Context(true)); @@ -51,9 +51,9 @@ static byte[] toByteArray(Object value, Coder coder) { * @param coder Coder to serialize with. * @return List of bytes representing serialized objects. */ - static List toByteArrays(Iterable values, final Coder coder) { + static List toByteArrays(Iterable values, final Coder coder) { List res = Lists.newLinkedList(); - for (Object value : values) { + for (T value : values) { res.add(toByteArray(value, coder)); } return res; 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 index 7e45c95da917..84c78d93a5e0 100644 --- 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 @@ -15,165 +15,183 @@ package com.cloudera.dataflow.spark.aggregators; -import java.io.Serializable; -import java.util.Map; -import java.util.TreeMap; - import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.Map; +import java.util.TreeMap; + /** - * What is an Agg? - * wrapper around a map of named aggregators. - * This allows us to add a named aggregator on the fly. - * we create an accumulable instance of aggs every time in the spark context. - * When the dataflow + * 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 instanyiate 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> mNamedAggregators = new TreeMap<>(); + /** Constructs a new NamedAggregators instance. */ + public NamedAggregators() { + } + + /** + * Constructs a new named aggrgators 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); + } + + /** + * Returns the value of the aggregator associated with the specified name. + * + * @param name Name of aggregator to retrieve. + * @param typeClass Type class to cast the value to. + * @param Type to be returned. + * @return + */ + public T getValue(String name, Class typeClass) { + return typeClass.cast(mNamedAggregators.get(name).render()); + } + + /** + * Merges another NamedAggregators instance with this instance. + * + * @param other The other instance of named aggragtors ot merge. + * @return This instance of Named aggragtors with associated states updated to reflect the + * other instance's aggregators. + */ + public NamedAggregators merge(NamedAggregators other) { + for (Map.Entry> 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 State merge(State s1, State s2) { + return ((State) s1).merge((State) s2); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (Map.Entry> e : mNamedAggregators.entrySet()) { + sb.append(e.getKey()).append(": ").append(e.getValue().render()); + } + return sb.toString(); + } + + /** + * @param Input data type + * @param Intermediate data type (useful for averages) + * @param Output datatype + */ + public interface State extends Serializable { /** - * Why is this final if you later add states to it? + * @param element */ - private final Map mNamedAggregators = new TreeMap<>(); + void update(In element); - public NamedAggregators() { - } + State merge(State other); - /** - * is "named" the label for a state? - * - * @param named - * @param state - */ - public NamedAggregators(String named, State state) { - this.mNamedAggregators.put(named, state); - } + Inter current(); - /** - * @param named - * @param typeClass - * @param - * @return - */ - public T getValue(String named, Class typeClass) { - return typeClass.cast(mNamedAggregators.get(named).render()); + Out render(); + } + + /** + * => combineFunction in data flow. + * + * @param + * @param + * @param + */ + public static class CombineFunctionState implements State { + + private Combine.CombineFn combineFn; + private Inter state; + + public CombineFunctionState(Combine.CombineFn combineFn) { + this.combineFn = combineFn; + this.state = combineFn.createAccumulator(); } - public NamedAggregators merge(NamedAggregators other) { - for (Map.Entry e : other.mNamedAggregators.entrySet()) { - State cur = mNamedAggregators.get(e.getKey()); - if (cur == null) { - mNamedAggregators.put(e.getKey(), e.getValue()); - } else { - mNamedAggregators.put(e.getKey(), cur.merge(e.getValue())); - } - } - return this; + @Override + public void update(In element) { + combineFn.addInput(state, element); } @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Map.Entry e : mNamedAggregators.entrySet()) { - sb.append(e.getKey()).append(": ").append(e.getValue().render()); - } - return sb.toString(); + public State merge(State other) { + this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); + return this; } - /** - * @param Input data type - * @param Intermediate data type (useful for averages) - * @param Output datatype - */ - public interface State extends Serializable { - /** - * @param element - */ - void update(In element); + @Override + public Inter current() { + return state; + } - State merge(State other); + @Override + public Out render() { + return combineFn.extractOutput(state); + } + } + + /** + * states correspond to dataflow objects. this one => serializable function + * + * @param + * @param + */ + public static class SerFunctionState implements State { + + private final SerializableFunction, Out> sfunc; + private Out state; + + public SerFunctionState(SerializableFunction, Out> sfunc) { + this.sfunc = sfunc; + this.state = sfunc.apply(ImmutableList.of()); + } - Inter current(); + @Override + public void update(In element) { + this.state = sfunc.apply(ImmutableList.of(element, (In) state)); + } - Out render(); + @Override + public State merge(State other) { + // Add exception catching and logging here. + this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); + return this; } - /** - * => combineFunction in data flow. - * - * @param - * @param - * @param - */ - public static class CombineFunctionState implements State { - - private Combine.CombineFn combineFn; - private Inter state; - - public CombineFunctionState(Combine.CombineFn combineFn) { - this.combineFn = combineFn; - this.state = combineFn.createAccumulator(); - } - - @Override - public void update(In element) { - combineFn.addInput(state, element); - } - - @Override - public State merge(State 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 Out current() { + return state; } - /** - * states correspond to dataflow objects. this one => seriazable function - * - * @param - * @param - */ - public static class SerFunctionState implements State { - - private SerializableFunction, Out> sfunc; - private Out state; - - public SerFunctionState(SerializableFunction, Out> sfunc) { - this.sfunc = sfunc; - this.state = sfunc.apply(ImmutableList.of()); - } - - @Override - public void update(In element) { - this.state = sfunc.apply(ImmutableList.of(element, (In) state)); - } - - @Override - public State merge(State other) { - // Add exception catching and logging here. - this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); - return this; - } - - @Override - public Out current() { - return state; - } - - @Override - public Out render() { - return state; - } + @Override + public Out render() { + return state; } + } } diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java index 6c73fa67102b..44d1c5099a3b 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java @@ -41,95 +41,98 @@ public class WordCountTest { - static TupleTag upper = new TupleTag<>(); - static TupleTag lower = new TupleTag<>(); - static TupleTag> lowerCnts = new TupleTag<>(); - static TupleTag> upperCnts = new TupleTag<>(); - - @Test - public void testRun() throws Exception { - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - PCollection regex = p.apply(Create.of("[^a-zA-Z']+")); - PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); - PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); - PCollectionList list = PCollectionList.of(w1).and(w2); - - PCollection union = list.apply(Flatten.create()); - PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); - PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); - - EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); - Iterable> actualLower = res.get(luc.get(lowerCnts)); - Iterable> actualUpper = res.get(luc.get(upperCnts)); - Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); - Iterable 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); + static TupleTag upper = new TupleTag<>(); + static TupleTag lower = new TupleTag<>(); + static TupleTag> lowerCnts = new TupleTag<>(); + static TupleTag> upperCnts = new TupleTag<>(); + + @Test + public void testRun() throws Exception { + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection regex = p.apply(Create.of("[^a-zA-Z']+")); + PCollection w1 = p.apply(Create.of("Here are some words to count", "and some others")); + PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); + PCollectionList list = PCollectionList.of(w1).and(w2); + + PCollection union = list.apply(Flatten.create()); + PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); + PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); + + EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + Iterable> actualLower = res.get(luc.get(lowerCnts)); + Iterable> actualUpper = res.get(luc.get(upperCnts)); + Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); + Iterable 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); + } + + /** + * A DoFn that tokenizes lines of text into individual words. + */ + static class ExtractWordsFn extends DoFn { + + Aggregator totalWords; + Aggregator maxWordLength; + PCollectionView regex; + + public ExtractWordsFn(PCollectionView regex) { + this.regex = regex; } - /** - * A DoFn that tokenizes lines of text into individual words. - */ - static class ExtractWordsFn extends DoFn { - - Aggregator totalWords; - Aggregator maxWordLength; - PCollectionView regex; - - public ExtractWordsFn(PCollectionView regex) { - this.regex = regex; - } - - @Override - public void startBundle(Context ctxt) { - this.totalWords = ctxt.createAggregator("totalWords", - new Sum.SumIntegerFn()); - this.maxWordLength = ctxt.createAggregator("maxWordLength", - new Max.MaxIntegerFn()); - } + @Override + public void startBundle(Context ctxt) { + this.totalWords = ctxt.createAggregator("totalWords", + new Sum.SumIntegerFn()); + this.maxWordLength = ctxt.createAggregator("maxWordLength", + new Max.MaxIntegerFn()); + } - @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); - } - } - } + @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, PCollectionTuple> { + public static class CountWords extends PTransform, PCollectionTuple> { - private final PCollectionView regex; + private final PCollectionView regex; - public CountWords(PCollectionView regex) { - this.regex = regex; - } + public CountWords(PCollectionView regex) { + this.regex = regex; + } - @Override - public PCollectionTuple apply(PCollection lines) { - // Convert lines of text into individual words. - PCollectionTuple lowerUpper = lines - .apply(ParDo.of(new ExtractWordsFn(regex)) - .withSideInputs(regex) - .withOutputTags(lower, TupleTagList.of(upper))); - lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); - lowerUpper.get(upper).setCoder(StringUtf8Coder.of()); - PCollection> lowerCounts = lowerUpper.get(lower).apply(Count.perElement()); - PCollection> upperCounts = lowerUpper.get(upper).apply(Count.perElement()); - return PCollectionTuple - .of(lowerCnts, lowerCounts) - .and(upperCnts, upperCounts); - } + @Override + public PCollectionTuple apply(PCollection lines) { + // Convert lines of text into individual words. + PCollectionTuple lowerUpper = lines + .apply(ParDo.of(new ExtractWordsFn(regex)) + .withSideInputs(regex) + .withOutputTags(lower, TupleTagList.of(upper))); + lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); + lowerUpper.get(upper).setCoder(StringUtf8Coder.of()); + PCollection> lowerCounts = lowerUpper.get(lower).apply(Count + .perElement()); + PCollection> upperCounts = lowerUpper.get(upper).apply(Count + .perElement()); + return PCollectionTuple + .of(lowerCnts, lowerCounts) + .and(upperCnts, upperCounts); } + } } From 74892633fa1c5a66f2f05cbce4b0e571ed72d71f Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 16 Dec 2014 13:15:05 -0800 Subject: [PATCH 024/137] Add simple word count test. --- runners/spark/README.md | 2 + ...est.java => MultiOutputWordCountTest.java} | 2 +- .../dataflow/spark/SimpleWordCountTest.java | 126 ++++++++++++++++++ 3 files changed, 129 insertions(+), 1 deletion(-) rename runners/spark/src/test/java/com/cloudera/dataflow/spark/{WordCountTest.java => MultiOutputWordCountTest.java} (99%) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java diff --git a/runners/spark/README.md b/runners/spark/README.md index 0921f4acd523..9ebdab1b11db 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -4,12 +4,14 @@ Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing and execute your pipeline in a program by calling `SparkPipelineRunner.run`. The Maven coordinates of the current version of this project are: + com.cloudera.dataflow.spark dataflow-spark 0.0.1 An example of running a pipeline against a spark cluster in local mode with 2 threads. + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); /** logic for building your pipeline */ EvaluationResult result = new SparkPipelineRunner("local[2]").run() diff --git a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java similarity index 99% rename from runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java rename to runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java index 44d1c5099a3b..9db4bde8ffc7 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/WordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -39,7 +39,7 @@ import org.junit.Assert; import org.junit.Test; -public class WordCountTest { +public class MultiOutputWordCountTest { static TupleTag upper = new TupleTag<>(); static TupleTag lower = new TupleTag<>(); 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 new file mode 100644 index 000000000000..c7eb0e26cf37 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -0,0 +1,126 @@ +/* + * 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.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.Sets; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +public class SimpleWordCountTest { + static final String[] WORDS_ARRAY = { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + static final List WORDS = Arrays.asList(WORDS_ARRAY); + static final Set EXPECTED_COUNT_SET = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", + "bob: 2"); + + @Test + public void testRun() throws Exception { + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + PCollection output = inputWords.apply(new CountWords()); + EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + Set actualCountSet = new HashSet(); + Iterator resultIter = res.get(output).iterator(); + while (resultIter.hasNext()) { + actualCountSet.add(resultIter.next()); + } + + Assert.assertTrue(String.format("Actual counts of words [%s] does not equal expected " + + "count[%s].", actualCountSet, EXPECTED_COUNT_SET), + EXPECTED_COUNT_SET.equals(actualCountSet)); + } + + /** + * A DoFn that tokenizes lines of text into individual words. + */ + static class ExtractWordsFn extends DoFn { + private Aggregator emptyLines; + + @Override + public void startBundle(Context c) { + emptyLines = c.createAggregator("emptyLines", new Sum.SumLongFn()); + } + + @Override + public void processElement(ProcessContext c) { + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // 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. + */ + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey() + ": " + c.element().getValue()); + } + } + + public static class CountWords extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + PCollection> wordCounts = + words.apply(Count.perElement()); + + // Format each word and count into a printable string. + PCollection results = wordCounts.apply( + ParDo.of(new FormatCountsFn())); + + return results; + } + + } +} From f9e8fab69223582ab5b68cdfb634fad4c0c525ef Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 16 Dec 2014 17:34:57 -0800 Subject: [PATCH 025/137] Factor out spark pipeline options. --- runners/spark/README.md | 20 ++-- .../dataflow/spark/SparkPipelineOptions.java | 13 +++ .../spark/SparkPipelineOptionsFactory.java | 9 ++ .../dataflow/spark/SparkPipelineRunner.java | 94 ++++++++----------- .../spark/MultiOutputWordCountTest.java | 2 +- .../dataflow/spark/SimpleWordCountTest.java | 2 +- .../TestSparkPipelineOptionsFactory.java | 19 ++++ 7 files changed, 97 insertions(+), 62 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java diff --git a/runners/spark/README.md b/runners/spark/README.md index 9ebdab1b11db..11ff192ffa14 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -1,7 +1,8 @@ spark-dataflow ============== -Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing a pipeline on a spark cluster is easy: Depend on spark-dataflow in your project -and execute your pipeline in a program by calling `SparkPipelineRunner.run`. +Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing a pipeline on a +spark cluster is easy: Depend on spark-dataflow in your project and execute your pipeline in a +program by calling `SparkPipelineRunner.run`. The Maven coordinates of the current version of this project are: @@ -9,11 +10,16 @@ The Maven coordinates of the current version of this project are: dataflow-spark 0.0.1 -An example of running a pipeline against a spark cluster in local mode with 2 -threads. +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: - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - /** logic for building your pipeline */ - EvaluationResult result = new SparkPipelineRunner("local[2]").run() + Pipeline p = + 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: + Pipeline p = + SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + options.setSparkMaster("spark://host:port"); + EvaluationResult result = SparkPipelineRunner.create(options).run(p); 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 new file mode 100644 index 000000000000..094b6c52ea63 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java @@ -0,0 +1,13 @@ +package com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + + +public interface SparkPipelineOptions extends PipelineOptions { + @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); +} 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 new file mode 100644 index 000000000000..ce6673ec1a95 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java @@ -0,0 +1,9 @@ +package com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; + +public class SparkPipelineOptionsFactory { + public static SparkPipelineOptions create() { + return PipelineOptionsFactory.create(SparkPipelineOptions.class); + } +} 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 index 4986db79e0f5..c70b362eaa00 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -15,90 +15,78 @@ package com.cloudera.dataflow.spark; -import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.io.AvroIO; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; -import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.transforms.Convert; -import com.google.cloud.dataflow.sdk.transforms.Create; -import com.google.cloud.dataflow.sdk.transforms.CreatePObject; -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.SeqDo; -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.PObject; -import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.PValue; -import com.google.cloud.dataflow.sdk.values.TupleTag; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -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.PairFunction; -import scala.Tuple2; -import java.lang.reflect.Field; -import java.util.Map; import java.util.logging.Logger; /** * 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 in Spark's local mode with two threads, we would do the following: + * a dataflow pipeline with the default options of a single threaded spark instance in local mode, + * we would do the following: * Pipeline p = - * EvaluationResult result = new SparkPipelineRunner("local[2]").run(p); + * 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: + * Pipeline p = + * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + * options.setSparkMaster("spark://host:port"); + * EvaluationResult result = SparkPipelineRunner.create(options).run(p); + * */ public class SparkPipelineRunner extends PipelineRunner { - private static final Logger LOG = - Logger.getLogger(SparkPipelineRunner.class.getName()); - /** The url of the spark master to connect to. */ - private final String master; + private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); + /** Options used in this pipeline runner.*/ + private 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); + } + /** * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single * thread. */ - public SparkPipelineRunner() { - this("local"); + private SparkPipelineRunner(SparkPipelineOptions options) { + mOptions = options; } - /** - * Constructor for a pipeline runner. - * - * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). - */ - public SparkPipelineRunner(String master) { - this.master = Preconditions.checkNotNull(master); - } @Override public EvaluationResult run(Pipeline pipeline) { - // TODO: get master from options - JavaSparkContext jsc = getContextFromOptions(pipeline.getOptions()); + JavaSparkContext jsc = getContext(); EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); return ctxt; } - private JavaSparkContext getContextFromOptions(PipelineOptions options) { - return new JavaSparkContext(master, options.getJobName()); + private JavaSparkContext getContext() { + return new JavaSparkContext(mOptions.getSparkMaster(), mOptions.getJobName()); } private static class Evaluator implements Pipeline.PipelineVisitor { 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 index 9db4bde8ffc7..a98d92d5ec97 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -59,7 +59,7 @@ public void testRun() throws Exception { PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); - EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + EvaluationResult res = SparkPipelineRunner.create().run(p); Iterable> actualLower = res.get(luc.get(lowerCnts)); Iterable> actualUpper = res.get(luc.get(upperCnts)); Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); 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 index c7eb0e26cf37..a3c546ccabec 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -50,7 +50,7 @@ public void testRun() throws Exception { Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(new CountWords()); - EvaluationResult res = new SparkPipelineRunner("local[2]").run(p); + EvaluationResult res = SparkPipelineRunner.create().run(p); Set actualCountSet = new HashSet(); Iterator resultIter = res.get(output).iterator(); while (resultIter.hasNext()) { 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 new file mode 100644 index 000000000000..7c2af428e0bc --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java @@ -0,0 +1,19 @@ +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(actualOptions.getSparkMaster(), "local[1]"); + } + + @Test + public void testSettingCustomOptions() { + SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create(); + actualOptions.setSparkMaster("spark://207.184.161.138:7077"); + Assert.assertEquals(actualOptions.getSparkMaster(), "spark://207.184.161.138:7077"); + } +} From ec172bac7b77774f86f0982d0261e1b7bb6d52e6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 19 Dec 2014 21:37:23 +0000 Subject: [PATCH 026/137] Miscellaneous inspection changes from IntelliJ --- .../dataflow/spark/BroadcastHelper.java | 3 +- .../cloudera/dataflow/spark/CoderHelpers.java | 12 ++-- .../cloudera/dataflow/spark/DoFnFunction.java | 2 +- .../dataflow/spark/EvaluationContext.java | 5 +- .../dataflow/spark/MultiDoFnFunction.java | 12 ++-- .../dataflow/spark/SparkPipelineOptions.java | 16 ++++- .../spark/SparkPipelineOptionsFactory.java | 20 +++++- .../dataflow/spark/SparkPipelineRunner.java | 2 +- .../dataflow/spark/SparkRuntimeContext.java | 5 +- .../dataflow/spark/TransformTranslator.java | 66 +++++++++++-------- .../spark/aggregators/NamedAggregators.java | 16 ++--- .../spark/MultiOutputWordCountTest.java | 10 +-- .../dataflow/spark/SimpleWordCountTest.java | 28 ++++---- .../TestSparkPipelineOptionsFactory.java | 4 +- 14 files changed, 126 insertions(+), 75 deletions(-) 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 index f0ee6ad9e4e4..8c728add5932 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -12,6 +12,7 @@ * the specific language governing permissions and limitations under the * License. */ + package com.cloudera.dataflow.spark; import java.io.ByteArrayInputStream; @@ -42,7 +43,7 @@ private T deserialize() { try { return coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); } catch (IOException e) { - throw new RuntimeException("Error deserializing broadcast variable", e); + throw new IllegalStateException("Error deserializing broadcast variable", e); } } } 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 index cd15f1a3501d..b6936e51cbb1 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -12,6 +12,7 @@ * the specific language governing permissions and limitations under the * License. */ + package com.cloudera.dataflow.spark; import java.io.ByteArrayInputStream; @@ -26,7 +27,10 @@ /** * Serialization utility class. */ -public class CoderHelpers { +public final class CoderHelpers { + private CoderHelpers() { + } + /** * Utility method for serializing an object using the specified coder. * @@ -39,7 +43,7 @@ static byte[] toByteArray(T value, Coder coder) { try { coder.encode(value, baos, new Coder.Context(true)); } catch (IOException e) { - throw new RuntimeException("Error encoding value: " + value, e); + throw new IllegalStateException("Error encoding value: " + value, e); } return baos.toByteArray(); } @@ -51,7 +55,7 @@ static byte[] toByteArray(T value, Coder coder) { * @param coder Coder to serialize with. * @return List of bytes representing serialized objects. */ - static List toByteArrays(Iterable values, final Coder coder) { + static List toByteArrays(Iterable values, Coder coder) { List res = Lists.newLinkedList(); for (T value : values) { res.add(toByteArray(value, coder)); @@ -72,7 +76,7 @@ static T fromByteArray(byte[] serialized, Coder coder) { try { return coder.decode(bais, new Coder.Context(true)); } catch (IOException e) { - throw new RuntimeException("Error decoding bytes for coder: " + coder, e); + throw new IllegalStateException("Error decoding bytes for coder: " + coder, e); } } 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 index 983a5959b4be..8cd593979ae8 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -79,7 +79,7 @@ public Iterable call(Iterator iter) throws Exception { private class ProcCtxt extends DoFn.ProcessContext { - private List outputs = new LinkedList<>(); + private final List outputs = new LinkedList<>(); private I element; public ProcCtxt(DoFn fn) { 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 index ae26ee9d9d4b..f992cf59a3e8 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -12,6 +12,7 @@ * the specific language governing permissions and limitations under the * License. */ + package com.cloudera.dataflow.spark; import java.util.List; @@ -111,7 +112,8 @@ BroadcastHelper getBroadcastHelper(PObject value) { public T get(PObject value) { if (pobjects.containsKey(value)) { return (T) pobjects.get(value); - } else if (rdds.containsKey(value)) { + } + if (rdds.containsKey(value)) { JavaRDDLike rdd = rdds.get(value); //TODO: need same logic from get() method below here for serialization of bytes T res = (T) Iterables.getOnlyElement(rdd.collect()); @@ -133,6 +135,7 @@ public Iterable get(PCollection pcollection) { JavaRDDLike bytes = rdd.map(CoderHelpers.toByteFunction(coder)); List clientBytes = bytes.collect(); return Iterables.transform(clientBytes, new Function() { + @Override public T apply(byte[] bytes) { return (T) CoderHelpers.fromByteArray(bytes, coder); } 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 index 362119666eb2..922ae04aed57 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -12,6 +12,7 @@ * the specific language governing permissions and limitations under the * License. */ + package com.cloudera.dataflow.spark; import java.util.Collection; @@ -36,8 +37,8 @@ import scala.Tuple2; /** - * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enrishing the - * undelrying data with multiple TupleTags. + * DoFunctions ignore side outputs. MultiDoFunctions deal with side outputs by enriching the + * underlying data with multiple TupleTags. * * @param Input type for DoFunction. * @param Output type for DoFunction. @@ -52,7 +53,7 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT private final TupleTag mMainOutputTag; private final Map, BroadcastHelper> mSideInputs; - public MultiDoFnFunction( + MultiDoFnFunction( DoFn fn, SparkRuntimeContext runtimeContext, TupleTag mainOutputTag, @@ -74,6 +75,7 @@ public Iterable, Object>> call(Iterator iter) throws Excep mFunction.finishBundle(ctxt); return Iterables.transform(ctxt.outputs.entries(), new Function, Object>, Tuple2, Object>>() { + @Override public Tuple2, Object> apply(Map.Entry, Object> input) { return new Tuple2, Object>(input.getKey(), input.getValue()); } @@ -82,10 +84,10 @@ public Tuple2, Object> apply(Map.Entry, Object> input) { private class ProcCtxt extends DoFn.ProcessContext { - private Multimap, Object> outputs = LinkedListMultimap.create(); + private final Multimap, Object> outputs = LinkedListMultimap.create(); private I element; - public ProcCtxt(DoFn fn) { + ProcCtxt(DoFn fn) { fn.super(); } 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 index 094b6c52ea63..d5d96afcbb0d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java @@ -1,10 +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 com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; - public interface SparkPipelineOptions extends PipelineOptions { @Description("The url of the spark master to connect to, (e.g. spark://host:port, local[4]).") @Default.String("local[1]") 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 index ce6673ec1a95..510038cc13ae 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java @@ -1,8 +1,26 @@ +/* + * 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 class SparkPipelineOptionsFactory { +public final class SparkPipelineOptionsFactory { + private SparkPipelineOptionsFactory() { + } + public static SparkPipelineOptions create() { return PipelineOptionsFactory.create(SparkPipelineOptions.class); } 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 index c70b362eaa00..0cdfa34072de 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -44,7 +44,7 @@ public class SparkPipelineRunner extends PipelineRunner { private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); /** Options used in this pipeline runner.*/ - private SparkPipelineOptions mOptions; + private final SparkPipelineOptions mOptions; /** * Creates and returns a new SparkPipelineRunner with default options. In particular, against a 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 index 7c1b46745287..82bf4491e9fe 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -12,6 +12,7 @@ * the specific language governing permissions and limitations under the * License. */ + package com.cloudera.dataflow.spark; import java.io.Serializable; @@ -37,11 +38,11 @@ class SparkRuntimeContext implements Serializable { /** * An accumulator that is a map from names to aggregators. */ - private Accumulator accum; + private final Accumulator accum; /** * Map fo names to dataflow aggregators. */ - private Map aggregators = new HashMap<>(); + private final Map aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); 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 index e163de2afd3f..a23d509e95a4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -1,3 +1,18 @@ +/* + * 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.api.client.util.Maps; @@ -22,19 +37,18 @@ /** * Supports translation between a DataFlow transform, and Spark's operations on RDDs. */ -public class TransformTranslator { +public final class TransformTranslator { + private TransformTranslator() { + } + private static class FieldGetter { - private Map fields; + private final Map fields; public FieldGetter(Class clazz) { this.fields = Maps.newHashMap(); for (Field f : clazz.getDeclaredFields()) { - try { - f.setAccessible(true); - this.fields.put(f.getName(), f); - } catch (Exception e) { - throw new RuntimeException(e); - } + f.setAccessible(true); + this.fields.put(f.getName(), f); } } @@ -42,12 +56,12 @@ public T get(String fieldname, Object value) { try { return (T) fields.get(fieldname).get(value); } catch (IllegalAccessException e) { - throw new RuntimeException(e); + throw new IllegalStateException(e); } } } - private static TransformEvaluator FLATTEN = new TransformEvaluator() { + private static final TransformEvaluator FLATTEN = new TransformEvaluator() { @Override public void evaluate(Flatten transform, EvaluationContext context) { PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); @@ -59,15 +73,15 @@ public void evaluate(Flatten transform, EvaluationContext context) { context.setOutputRDD(transform, rdd); } }; - private static TransformEvaluator GBK = new TransformEvaluator() { + private static final TransformEvaluator GBK = new TransformEvaluator() { @Override public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); } }; - private static FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); - private static TransformEvaluator GROUPED = new TransformEvaluator() { + private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); + private static final TransformEvaluator GROUPED = new TransformEvaluator() { @Override public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); @@ -102,7 +116,7 @@ public Object call(Object o) throws Exception { } - private static TransformEvaluator PARDO = new TransformEvaluator() { + private static final TransformEvaluator PARDO = new TransformEvaluator() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { DoFnFunction dofn = new DoFnFunction(transform.getFn(), @@ -112,8 +126,8 @@ public void evaluate(ParDo.Bound transform, EvaluationContext context) { } }; - private static FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); - private static TransformEvaluator MULTIDO = new TransformEvaluator() { + private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); + private static final TransformEvaluator MULTIDO = new TransformEvaluator() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { MultiDoFnFunction multifn = new MultiDoFnFunction( @@ -136,7 +150,7 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { }; - private static TransformEvaluator READ_TEXT = new TransformEvaluator() { + private static final TransformEvaluator READ_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); @@ -145,7 +159,7 @@ public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { } }; - private static TransformEvaluator WRITE_TEXT = new TransformEvaluator() { + private static final TransformEvaluator WRITE_TEXT = new TransformEvaluator() { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { JavaRDDLike last = context.getInputRDD(transform); @@ -154,7 +168,7 @@ public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { } }; - private static TransformEvaluator READ_AVRO = new TransformEvaluator() { + private static final TransformEvaluator READ_AVRO = new TransformEvaluator() { @Override public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); @@ -163,7 +177,7 @@ public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { } }; - private static TransformEvaluator WRITE_AVRO = new TransformEvaluator() { + private static final TransformEvaluator WRITE_AVRO = new TransformEvaluator() { @Override public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { JavaRDDLike last = context.getInputRDD(transform); @@ -172,7 +186,7 @@ public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { } }; - private static TransformEvaluator CREATE = new TransformEvaluator() { + private static final TransformEvaluator CREATE = new TransformEvaluator() { @Override public void evaluate(Create transform, EvaluationContext context) { Iterable elems = transform.getElements(); @@ -183,14 +197,14 @@ public void evaluate(Create transform, EvaluationContext context) { } }; - private static TransformEvaluator CREATE_POBJ = new TransformEvaluator() { + private static final TransformEvaluator CREATE_POBJ = new TransformEvaluator() { @Override public void evaluate(CreatePObject transform, EvaluationContext context) { context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); } }; - private static TransformEvaluator TO_ITER = new TransformEvaluator() { + private static final TransformEvaluator TO_ITER = new TransformEvaluator() { @Override public void evaluate(Convert.ToIterable transform, EvaluationContext context) { PCollection in = (PCollection) context.getInput(transform); @@ -199,7 +213,7 @@ public void evaluate(Convert.ToIterable transform, EvaluationContext context) { } }; - private static TransformEvaluator TO_ITER_WIN = + private static final TransformEvaluator TO_ITER_WIN = new TransformEvaluator() { @Override public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { @@ -216,7 +230,7 @@ public WindowedValue apply(Object o) { }; private static class TupleTagFilter implements Function, Boolean> { - private TupleTag tag; + private final TupleTag tag; public TupleTagFilter(TupleTag tag) { this.tag = tag; @@ -228,7 +242,7 @@ public Boolean call(Tuple2 input) throws Exception { } } - private static TransformEvaluator SEQDO = new TransformEvaluator() { + private static final TransformEvaluator SEQDO = new TransformEvaluator() { @Override public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { PObjectValueTuple inputValues = context.getPObjectTuple(transform); 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 index 84c78d93a5e0..e372b951a7c1 100644 --- 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 @@ -26,7 +26,7 @@ /** * 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 instanyiate in the the spark context before the job + * 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 { @@ -37,7 +37,7 @@ public NamedAggregators() { } /** - * Constructs a new named aggrgators instance that contains a mapping from the specified + * Constructs a new named aggregators instance that contains a mapping from the specified * `named` to the associated initial state. * * @param name Name of aggregator. @@ -48,12 +48,10 @@ public NamedAggregators(String name, State state) { } /** - * Returns the value of the aggregator associated with the specified name. - * * @param name Name of aggregator to retrieve. * @param typeClass Type class to cast the value to. * @param Type to be returned. - * @return + * @return the value of the aggregator associated with the specified name */ public T getValue(String name, Class typeClass) { return typeClass.cast(mNamedAggregators.get(name).render()); @@ -62,8 +60,8 @@ public T getValue(String name, Class typeClass) { /** * Merges another NamedAggregators instance with this instance. * - * @param other The other instance of named aggragtors ot merge. - * @return This instance of Named aggragtors with associated states updated to reflect the + * @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) { @@ -105,7 +103,7 @@ public String toString() { */ public interface State extends Serializable { /** - * @param element + * @param element new element to update state */ void update(In element); @@ -125,7 +123,7 @@ public interface State extends Serializable { */ public static class CombineFunctionState implements State { - private Combine.CombineFn combineFn; + private final Combine.CombineFn combineFn; private Inter state; public CombineFunctionState(Combine.CombineFn combineFn) { 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 index a98d92d5ec97..ac884dd4c4ef 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -41,10 +41,10 @@ public class MultiOutputWordCountTest { - static TupleTag upper = new TupleTag<>(); - static TupleTag lower = new TupleTag<>(); - static TupleTag> lowerCnts = new TupleTag<>(); - static TupleTag> upperCnts = new TupleTag<>(); + private static final TupleTag upper = new TupleTag<>(); + private static final TupleTag lower = new TupleTag<>(); + private static final TupleTag> lowerCnts = new TupleTag<>(); + private static final TupleTag> upperCnts = new TupleTag<>(); @Test public void testRun() throws Exception { @@ -78,7 +78,7 @@ static class ExtractWordsFn extends DoFn { Aggregator totalWords; Aggregator maxWordLength; - PCollectionView regex; + final PCollectionView regex; public ExtractWordsFn(PCollectionView regex) { this.regex = regex; 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 index a3c546ccabec..7c6fd68dd85d 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -33,16 +33,15 @@ import java.util.Arrays; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Set; public class SimpleWordCountTest { - static final String[] WORDS_ARRAY = { + private static final String[] WORDS_ARRAY = { "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"}; - static final List WORDS = Arrays.asList(WORDS_ARRAY); - static final Set EXPECTED_COUNT_SET = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", + private static final List WORDS = Arrays.asList(WORDS_ARRAY); + private static final Set EXPECTED_COUNT_SET = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2"); @Test @@ -51,15 +50,14 @@ public void testRun() throws Exception { PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(new CountWords()); EvaluationResult res = SparkPipelineRunner.create().run(p); - Set actualCountSet = new HashSet(); - Iterator resultIter = res.get(output).iterator(); - while (resultIter.hasNext()) { - actualCountSet.add(resultIter.next()); + Set actualCountSet = new HashSet<>(); + for (String s : res.get(output)) { + actualCountSet.add(s); } - Assert.assertTrue(String.format("Actual counts of words [%s] does not equal expected " + - "count[%s].", actualCountSet, EXPECTED_COUNT_SET), - EXPECTED_COUNT_SET.equals(actualCountSet)); + Assert.assertEquals(String.format("Actual counts of words [%s] does not equal expected count[%s].", + actualCountSet, EXPECTED_COUNT_SET), + EXPECTED_COUNT_SET, actualCountSet); } /** @@ -96,14 +94,14 @@ public void processElement(ProcessContext c) { /** * A DoFn that converts a Word and Count into a printable string. */ - static class FormatCountsFn extends DoFn, String> { + private static class FormatCountsFn extends DoFn, String> { @Override public void processElement(ProcessContext c) { c.output(c.element().getKey() + ": " + c.element().getValue()); } } - public static class CountWords extends PTransform, PCollection> { + private static class CountWords extends PTransform, PCollection> { @Override public PCollection apply(PCollection lines) { @@ -116,10 +114,8 @@ public PCollection apply(PCollection lines) { words.apply(Count.perElement()); // Format each word and count into a printable string. - PCollection results = wordCounts.apply( - ParDo.of(new FormatCountsFn())); - return results; + return wordCounts.apply(ParDo.of(new FormatCountsFn())); } } 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 index 7c2af428e0bc..991cdedab5b4 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TestSparkPipelineOptionsFactory.java @@ -7,13 +7,13 @@ public class TestSparkPipelineOptionsFactory { @Test public void testDefaultCreateMethod() { SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create(); - Assert.assertEquals(actualOptions.getSparkMaster(), "local[1]"); + Assert.assertEquals("local[1]", actualOptions.getSparkMaster()); } @Test public void testSettingCustomOptions() { SparkPipelineOptions actualOptions = SparkPipelineOptionsFactory.create(); actualOptions.setSparkMaster("spark://207.184.161.138:7077"); - Assert.assertEquals(actualOptions.getSparkMaster(), "spark://207.184.161.138:7077"); + Assert.assertEquals("spark://207.184.161.138:7077", actualOptions.getSparkMaster()); } } From ed1e2f7fb66f0746c1a8511b398f41a6df0e1513 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 20 Dec 2014 08:36:13 +0000 Subject: [PATCH 027/137] Issue #13 : attempt to remove all generics warnings, or handle them explicitly --- runners/spark/pom.xml | 3 + .../cloudera/dataflow/spark/DoFnFunction.java | 8 +- .../dataflow/spark/EvaluationContext.java | 54 ++- .../dataflow/spark/MultiDoFnFunction.java | 12 +- .../dataflow/spark/SparkPipelineRunner.java | 9 +- .../dataflow/spark/SparkRuntimeContext.java | 17 +- .../dataflow/spark/TransformTranslator.java | 391 ++++++++++-------- .../spark/aggregators/NamedAggregators.java | 12 +- 8 files changed, 303 insertions(+), 203 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 3d4afdf938f0..5966c64fd725 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -32,6 +32,9 @@ License. 1.7 1.7 + + -Xlint:all,-serial + 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 index 8cd593979ae8..673237e574e8 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -64,7 +64,7 @@ public DoFnFunction( @Override public Iterable call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt<>(mFunction); + ProcCtxt ctxt = new ProcCtxt(mFunction); //setup mFunction.startBundle(ctxt); //operation @@ -77,7 +77,7 @@ public Iterable call(Iterator iter) throws Exception { return ctxt.outputs; } - private class ProcCtxt extends DoFn.ProcessContext { + private class ProcCtxt extends DoFn.ProcessContext { private final List outputs = new LinkedList<>(); private I element; @@ -93,7 +93,9 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - return (T) mSideInputs.get(view.getTagInternal()).getValue(); + @SuppressWarnings("unchecked") + T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); + return value; } @Override 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 index f992cf59a3e8..4105ab61eb1f 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -45,9 +45,9 @@ public class EvaluationContext implements EvaluationResult { private final JavaSparkContext jsc; private final Pipeline pipeline; private final SparkRuntimeContext runtime; - private final Map rdds = Maps.newHashMap(); + private final Map> rdds = Maps.newHashMap(); private final Set multireads = Sets.newHashSet(); - private final Map pobjects = Maps.newHashMap(); + private final Map, Object> pobjects = Maps.newHashMap(); public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; @@ -68,23 +68,27 @@ SparkRuntimeContext getRuntimeContext() { } I getInput(PTransform transform) { - return (I) pipeline.getInput(transform); + @SuppressWarnings("unchecked") + I input = (I) pipeline.getInput(transform); + return input; } O getOutput(PTransform transform) { - return (O) pipeline.getOutput(transform); + @SuppressWarnings("unchecked") + O output = (O) pipeline.getOutput(transform); + return output; } - void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + void setOutputRDD(PTransform transform, JavaRDDLike rdd) { rdds.put((PValue) getOutput(transform), rdd); } - void setPObjectValue(PObject pobject, Object value) { + void setPObjectValue(PObject pobject, Object value) { pobjects.put(pobject, value); } - JavaRDDLike getRDD(PValue pvalue) { - JavaRDDLike rdd = rdds.get(pvalue); + JavaRDDLike getRDD(PValue pvalue) { + JavaRDDLike rdd = rdds.get(pvalue); if (multireads.contains(pvalue)) { // Ensure the RDD is marked as cached rdd.rdd().cache(); @@ -94,11 +98,11 @@ JavaRDDLike getRDD(PValue pvalue) { return rdd; } - void setRDD(PValue pvalue, JavaRDDLike rdd) { + void setRDD(PValue pvalue, JavaRDDLike rdd) { rdds.put(pvalue, rdd); } - JavaRDDLike getInputRDD(PTransform transform) { + JavaRDDLike getInputRDD(PTransform transform) { return getRDD((PValue) pipeline.getInput(transform)); } @@ -111,11 +115,14 @@ BroadcastHelper getBroadcastHelper(PObject value) { @Override public T get(PObject value) { if (pobjects.containsKey(value)) { - return (T) pobjects.get(value); + @SuppressWarnings("unchecked") + T result = (T) pobjects.get(value); + return result; } if (rdds.containsKey(value)) { - JavaRDDLike rdd = rdds.get(value); + JavaRDDLike rdd = rdds.get(value); //TODO: need same logic from get() method below here for serialization of bytes + @SuppressWarnings("unchecked") T res = (T) Iterables.getOnlyElement(rdd.collect()); pobjects.put(value, res); return res; @@ -130,10 +137,11 @@ public T getAggregatorValue(String named, Class resultType) { @Override public Iterable get(PCollection pcollection) { - JavaRDDLike rdd = getRDD(pcollection); - final Coder coder = pcollection.getCoder(); - JavaRDDLike bytes = rdd.map(CoderHelpers.toByteFunction(coder)); - List clientBytes = bytes.collect(); + @SuppressWarnings("unchecked") + JavaRDDLike rdd = (JavaRDDLike) getRDD(pcollection); + final Coder coder = pcollection.getCoder(); + JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); + List clientBytes = bytesRDD.collect(); return Iterables.transform(clientBytes, new Function() { @Override public T apply(byte[] bytes) { @@ -142,16 +150,24 @@ public T apply(byte[] bytes) { }); } - PObjectValueTuple getPObjectTuple(PTransform transform) { + PObjectValueTuple getPObjectTuple(PTransform transform) { PObjectTuple pot = (PObjectTuple) pipeline.getInput(transform); PObjectValueTuple povt = PObjectValueTuple.empty(); for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - povt = povt.and((TupleTag) e.getKey(), get(e.getValue())); + povt = and(povt, e); } return povt; } - void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { + private PObjectValueTuple and(PObjectValueTuple povt, Map.Entry, PObject> e) { + @SuppressWarnings("unchecked") + TupleTag ttKey = (TupleTag) e.getKey(); + @SuppressWarnings("unchecked") + PObject potValue = (PObject) e.getValue(); + return povt.and(ttKey, get(potValue)); + } + + void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); for (Map.Entry, PObject> e : pot.getAll().entrySet()) { pobjects.put(e.getValue(), outputValues.get(e.getKey())); 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 index 922ae04aed57..70facf82bb4d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -50,7 +50,7 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; - private final TupleTag mMainOutputTag; + private final TupleTag mMainOutputTag; private final Map, BroadcastHelper> mSideInputs; MultiDoFnFunction( @@ -61,12 +61,12 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT this.mFunction = fn; this.mRuntimeContext = runtimeContext; this.mMainOutputTag = mainOutputTag; - this. mSideInputs = sideInputs; + this.mSideInputs = sideInputs; } @Override public Iterable, Object>> call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(mFunction); + ProcCtxt ctxt = new ProcCtxt(mFunction); mFunction.startBundle(ctxt); while (iter.hasNext()) { ctxt.element = iter.next(); @@ -82,7 +82,7 @@ public Tuple2, Object> apply(Map.Entry, Object> input) { }); } - private class ProcCtxt extends DoFn.ProcessContext { + private class ProcCtxt extends DoFn.ProcessContext { private final Multimap, Object> outputs = LinkedListMultimap.create(); private I element; @@ -98,7 +98,9 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - return (T) mSideInputs.get(view.getTagInternal()).getValue(); + @SuppressWarnings("unchecked") + T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); + return value; } @Override 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 index 0cdfa34072de..f8b261a125ee 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -107,8 +107,13 @@ public void leaveCompositeTransform(TransformTreeNode node) { @Override public void visitTransform(TransformTreeNode node) { - PTransform transform = node.getTransform(); - TransformEvaluator evaluator = TransformTranslator.getTransformEvaluator(transform.getClass()); + doVisitTransform(node.getTransform()); + } + + private void doVisitTransform(PT transform) { + @SuppressWarnings("unchecked") + TransformEvaluator evaluator = (TransformEvaluator) + TransformTranslator.getTransformEvaluator(transform.getClass()); LOG.info("Evaluating " + transform); evaluator.evaluate(transform, ctxt); } 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 index 82bf4491e9fe..5b1e085d9a67 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -42,7 +42,7 @@ class SparkRuntimeContext implements Serializable { /** * Map fo names to dataflow aggregators. */ - private final Map aggregators = new HashMap<>(); + private final Map> aggregators = new HashMap<>(); public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); @@ -77,12 +77,13 @@ public synchronized PipelineOptions getPipelineOptions() { public synchronized Aggregator createAggregator( String named, SerializableFunction, Out> sfunc) { - Aggregator aggregator = aggregators.get(named); + @SuppressWarnings("unchecked") + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { NamedAggregators.SerFunctionState state = new NamedAggregators .SerFunctionState<>(sfunc); accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator(state); + aggregator = new SparkAggregator<>(state); aggregators.put(named, aggregator); } return aggregator; @@ -100,12 +101,14 @@ public synchronized Aggregator createAggregator( public synchronized Aggregator createAggregator( String named, Combine.CombineFn combineFn) { - Aggregator aggregator = aggregators.get(named); + @SuppressWarnings("unchecked") + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { - NamedAggregators.CombineFunctionState state = new NamedAggregators - .CombineFunctionState<>(combineFn); + @SuppressWarnings("unchecked") + NamedAggregators.CombineFunctionState state = new NamedAggregators + .CombineFunctionState<>((Combine.CombineFn) combineFn); accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator(state); + aggregator = new SparkAggregator<>(state); aggregators.put(named, aggregator); } return aggregator; 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 index a23d509e95a4..477a8441bebe 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -19,9 +19,23 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Convert; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.CreatePObject; +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.SeqDo; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.values.*; +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.PObjectValueTuple; +import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; @@ -44,7 +58,7 @@ private TransformTranslator() { private static class FieldGetter { private final Map fields; - public FieldGetter(Class clazz) { + FieldGetter(Class clazz) { this.fields = Maps.newHashMap(); for (Field f : clazz.getDeclaredFields()) { f.setAccessible(true); @@ -54,202 +68,250 @@ public FieldGetter(Class clazz) { public T get(String fieldname, Object value) { try { - return (T) fields.get(fieldname).get(value); + @SuppressWarnings("unchecked") + T fieldValue = (T) fields.get(fieldname).get(value); + return fieldValue; } catch (IllegalAccessException e) { throw new IllegalStateException(e); } } } - private static final TransformEvaluator FLATTEN = new TransformEvaluator() { - @Override - public void evaluate(Flatten transform, EvaluationContext context) { - PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); - JavaRDD[] rdds = new JavaRDD[pcs.size()]; - for (int i = 0; i < rdds.length; i++) { - rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + private static TransformEvaluator> flatten() { + return new TransformEvaluator>() { + @SuppressWarnings("unchecked") + @Override + public void evaluate(Flatten transform, EvaluationContext context) { + PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + JavaRDD[] rdds = new JavaRDD[pcs.size()]; + for (int i = 0; i < rdds.length; i++) { + rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + } + JavaRDD rdd = context.getSparkContext().union(rdds); + context.setOutputRDD(transform, rdd); } - JavaRDD rdd = context.getSparkContext().union(rdds); - context.setOutputRDD(transform, rdd); - } - }; - private static final TransformEvaluator GBK = new TransformEvaluator() { - @Override - public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { - context.setOutputRDD(transform, fromPair(toPair(context.getInputRDD(transform)).groupByKey())); - } - }; + }; + } + + private static TransformEvaluator> gbk() { + return new TransformEvaluator>() { + @Override + public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike,?> inRDD = + (JavaRDDLike,?>) context.getInputRDD(transform); + context.setOutputRDD(transform, fromPair(toPair(inRDD).groupByKey())); + } + }; + } private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); - private static final TransformEvaluator GROUPED = new TransformEvaluator() { + private static TransformEvaluator> grouped() { + return new TransformEvaluator>() { + @Override + public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { + Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + @SuppressWarnings("unchecked") + JavaRDDLike>,?> inRDD = + (JavaRDDLike>,?>) context.getInputRDD(transform); + context.setOutputRDD(transform, inRDD.map(new KVFunction<>(keyed))); + } + }; + } + + private static final class KVFunction implements Function>, KV> { + private final Combine.KeyedCombineFn keyed; + + KVFunction(Combine.KeyedCombineFn keyed) { + this.keyed = keyed; + } + @Override - public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - final Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); - context.setOutputRDD(transform, context.getInputRDD(transform).map(new Function() { - @Override - public Object call(Object input) throws Exception { - KV kv = (KV) input; - return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); - } - })); + public KV call(KV> kv) throws Exception { + return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); } - }; + } - private static JavaPairRDD toPair(JavaRDDLike rdd) { - return rdd.mapToPair(new PairFunction() { + private static JavaPairRDD toPair(JavaRDDLike,?> rdd) { + return rdd.mapToPair(new PairFunction,K,V>() { @Override - public Tuple2 call(Object o) throws Exception { - KV kv = (KV) o; - return new Tuple2(kv.getKey(), kv.getValue()); + public Tuple2 call(KV kv) { + return new Tuple2<>(kv.getKey(), kv.getValue()); } }); } - private static JavaRDDLike fromPair(JavaPairRDD rdd) { - return rdd.map(new Function() { + private static JavaRDDLike,?> fromPair(JavaPairRDD rdd) { + return rdd.map(new Function,KV>() { @Override - public Object call(Object o) throws Exception { - Tuple2 t2 = (Tuple2) o; + public KV call(Tuple2 t2) { return KV.of(t2._1(), t2._2()); } }); } - - private static final TransformEvaluator PARDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.Bound transform, EvaluationContext context) { - DoFnFunction dofn = new DoFnFunction(transform.getFn(), - context.getRuntimeContext(), - getSideInputs(transform.getSideInputs(), context)); - context.setOutputRDD(transform, context.getInputRDD(transform).mapPartitions(dofn)); - } - }; + private static TransformEvaluator> parDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + DoFnFunction dofn = + new DoFnFunction<>(transform.getFn(), + context.getRuntimeContext(), + getSideInputs(transform.getSideInputs(), context)); + @SuppressWarnings("unchecked") + JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); + context.setOutputRDD(transform, inRDD.mapPartitions(dofn)); + } + }; + } private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); - private static final TransformEvaluator MULTIDO = new TransformEvaluator() { - @Override - public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - MultiDoFnFunction multifn = new MultiDoFnFunction( + private static TransformEvaluator> multiDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + TupleTag mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform); + MultiDoFnFunction multifn = new MultiDoFnFunction<>( transform.getFn(), context.getRuntimeContext(), - (TupleTag) MULTIDO_FG.get("mainOutputTag", transform), + mainOutputTag, getSideInputs(transform.getSideInputs(), context)); - JavaPairRDD all = context.getInputRDD(transform) + @SuppressWarnings("unchecked") + JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); + JavaPairRDD, Object> all = inRDD .mapPartitionsToPair(multifn) .cache(); - PCollectionTuple pct = (PCollectionTuple) context.getOutput(transform); - for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { - TupleTagFilter filter = new TupleTagFilter(e.getKey()); - JavaPairRDD filtered = all.filter(filter); - context.setRDD(e.getValue(), filtered.values()); + PCollectionTuple pct = context.getOutput(transform); + for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { + @SuppressWarnings("unchecked") + JavaPairRDD, Object> filtered = + all.filter(new TupleTagFilter(e.getKey())); + context.setRDD(e.getValue(), filtered.values()); + } } - } - }; + }; + } - private static final TransformEvaluator READ_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; + private static TransformEvaluator> readText() { + return new TransformEvaluator>() { + @Override + public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + } - private static final TransformEvaluator WRITE_TEXT = new TransformEvaluator() { - @Override - public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; + private static TransformEvaluator> writeText() { + return new TransformEvaluator>() { + @Override + public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + } - private static final TransformEvaluator READ_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; + private static TransformEvaluator> readAvro() { + return new TransformEvaluator>() { + @Override + public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + } - private static final TransformEvaluator WRITE_AVRO = new TransformEvaluator() { - @Override - public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - JavaRDDLike last = context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; + private static TransformEvaluator> writeAvro() { + return new TransformEvaluator>() { + @Override + public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + } - private static final TransformEvaluator CREATE = new TransformEvaluator() { - @Override - public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = transform.getElements(); - Coder coder = ((PCollection) context.getOutput(transform)).getCoder(); - JavaRDD rdd = context.getSparkContext().parallelize( + private static TransformEvaluator> create() { + return new TransformEvaluator>() { + @Override + public void evaluate(Create transform, EvaluationContext context) { + Iterable elems = transform.getElements(); + Coder coder = context.getOutput(transform).getCoder(); + JavaRDD rdd = context.getSparkContext().parallelize( CoderHelpers.toByteArrays(elems, coder)); - context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); - } - }; + context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + } - private static final TransformEvaluator CREATE_POBJ = new TransformEvaluator() { - @Override - public void evaluate(CreatePObject transform, EvaluationContext context) { - context.setPObjectValue((PObject) context.getOutput(transform), transform.getElement()); - } - }; + private static TransformEvaluator> createPObj() { + return new TransformEvaluator>() { + @Override + public void evaluate(CreatePObject transform, EvaluationContext context) { + context.setPObjectValue(context.getOutput(transform), transform.getElement()); + } + }; + } - private static final TransformEvaluator TO_ITER = new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterable transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, context.get(in)); - } - }; - - private static final TransformEvaluator TO_ITER_WIN = - new TransformEvaluator() { - @Override - public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { - PCollection in = (PCollection) context.getInput(transform); - PObject out = (PObject) context.getOutput(transform); - context.setPObjectValue(out, Iterables.transform(context.get(in), - new com.google.common.base.Function() { - @Override - public WindowedValue apply(Object o) { - return WindowedValue.valueInGlobalWindow(o); - } - })); - } - }; + private static TransformEvaluator> toIter() { + return new TransformEvaluator>() { + @Override + public void evaluate(Convert.ToIterable transform, EvaluationContext context) { + context.setPObjectValue(context.getOutput(transform), + context.get(context.getInput(transform))); + } + }; + } - private static class TupleTagFilter implements Function, Boolean> { - private final TupleTag tag; + private static TransformEvaluator> toIterWin() { + return new TransformEvaluator>() { + @Override + public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { + context.setPObjectValue(context.getOutput(transform), + Iterables.transform(context.get(context.getInput(transform)), + new com.google.common.base.Function>() { + @Override + public WindowedValue apply(T t) { + return WindowedValue.valueInGlobalWindow(t); + } + })); + } + }; + } - public TupleTagFilter(TupleTag tag) { + private static class TupleTagFilter implements Function, Object>, Boolean> { + private final TupleTag tag; + + private TupleTagFilter(TupleTag tag) { this.tag = tag; } @Override - public Boolean call(Tuple2 input) throws Exception { + public Boolean call(Tuple2, Object> input) { return tag.equals(input._1()); } } - private static final TransformEvaluator SEQDO = new TransformEvaluator() { - @Override - public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { - PObjectValueTuple inputValues = context.getPObjectTuple(transform); - PObjectValueTuple outputValues = transform.getFn().process(inputValues); - context.setPObjectTuple(transform, outputValues); - } - }; + private static TransformEvaluator seqDo() { + return new TransformEvaluator() { + @Override + public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { + PObjectValueTuple inputValues = context.getPObjectTuple(transform); + PObjectValueTuple outputValues = transform.getFn().process(inputValues); + context.setPObjectTuple(transform, outputValues); + } + }; + } private static Map, BroadcastHelper> getSideInputs( Iterable> views, @@ -265,26 +327,27 @@ private static Map, BroadcastHelper> getSideInputs( } } - private static final Map, TransformEvaluator> mEvaluators = Maps.newHashMap(); + private static final Map, TransformEvaluator> mEvaluators = Maps.newHashMap(); static { - mEvaluators.put(TextIO.Read.Bound.class, READ_TEXT); - mEvaluators.put(TextIO.Write.Bound.class, WRITE_TEXT); - mEvaluators.put(AvroIO.Read.Bound.class, READ_AVRO); - mEvaluators.put(AvroIO.Write.Bound.class, WRITE_AVRO); - mEvaluators.put(ParDo.Bound.class, PARDO); - mEvaluators.put(ParDo.BoundMulti.class, MULTIDO); - mEvaluators.put(SeqDo.BoundMulti.class, SEQDO); - mEvaluators.put(GroupByKey.GroupByKeyOnly.class, GBK); - mEvaluators.put(Combine.GroupedValues.class, GROUPED); - mEvaluators.put(Flatten.class, FLATTEN); - mEvaluators.put(Create.class, CREATE); - mEvaluators.put(CreatePObject.class, CREATE_POBJ); - mEvaluators.put(Convert.ToIterable.class, TO_ITER); - mEvaluators.put(Convert.ToIterableWindowedValue.class, TO_ITER_WIN); + mEvaluators.put(TextIO.Read.Bound.class, readText()); + mEvaluators.put(TextIO.Write.Bound.class, writeText()); + mEvaluators.put(AvroIO.Read.Bound.class, readAvro()); + mEvaluators.put(AvroIO.Write.Bound.class, writeAvro()); + mEvaluators.put(ParDo.Bound.class, parDo()); + mEvaluators.put(ParDo.BoundMulti.class, multiDo()); + mEvaluators.put(SeqDo.BoundMulti.class, seqDo()); + mEvaluators.put(GroupByKey.GroupByKeyOnly.class, gbk()); + mEvaluators.put(Combine.GroupedValues.class, grouped()); + mEvaluators.put(Flatten.class, flatten()); + mEvaluators.put(Create.class, create()); + mEvaluators.put(CreatePObject.class, createPObj()); + mEvaluators.put(Convert.ToIterable.class, toIter()); + mEvaluators.put(Convert.ToIterableWindowedValue.class, toIterWin()); } - public static TransformEvaluator getTransformEvaluator(Class clazz) { - TransformEvaluator transform = mEvaluators.get(clazz); + public static TransformEvaluator getTransformEvaluator(Class clazz) { + @SuppressWarnings("unchecked") + TransformEvaluator transform = (TransformEvaluator) mEvaluators.get(clazz); if (transform == null) { throw new IllegalStateException("No TransformEvaluator registered for " + clazz); } 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 index e372b951a7c1..06ee4d37c06e 100644 --- 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 @@ -171,13 +171,19 @@ public SerFunctionState(SerializableFunction, Out> sfunc) { @Override public void update(In element) { - this.state = sfunc.apply(ImmutableList.of(element, (In) state)); + @SuppressWarnings("unchecked") + In thisState = (In) state; + this.state = sfunc.apply(ImmutableList.of(element, thisState)); } @Override - public State merge(State other) { + public State merge(State other) { // Add exception catching and logging here. - this.state = sfunc.apply(ImmutableList.of((In) state, (In) other.current())); + @SuppressWarnings("unchecked") + In thisState = (In) state; + @SuppressWarnings("unchecked") + In otherCurrent = (In) other.current(); + this.state = sfunc.apply(ImmutableList.of(thisState, otherCurrent)); return this; } From 225f6c0121e6171b02097cac86724920e0e32b50 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 21 Dec 2014 18:31:44 +0000 Subject: [PATCH 028/137] Update and specify POM plugin config; Update Spark to 1.1.1, JUnit to 4.12, Spark 1.2; Add source, javadoc plugins and other info; Fix javadoc errors and a few typos --- runners/spark/pom.xml | 171 ++++++++++++++++-- .../cloudera/dataflow/spark/CoderHelpers.java | 4 +- .../cloudera/dataflow/spark/DoFnFunction.java | 7 +- .../dataflow/spark/EvaluationContext.java | 10 +- .../dataflow/spark/EvaluationResult.java | 8 +- .../dataflow/spark/SparkPipelineRunner.java | 4 +- .../dataflow/spark/SparkRuntimeContext.java | 4 +- .../spark/aggregators/NamedAggregators.java | 13 +- .../spark/MultiOutputWordCountTest.java | 8 +- .../dataflow/spark/SimpleWordCountTest.java | 6 +- 10 files changed, 180 insertions(+), 55 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 5966c64fd725..aece8908b55c 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,29 +22,119 @@ License. UTF-8 + UTF-8 + 1.7 + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + ${java.version} + ${java.version} + + -Xlint:all,-serial + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.18 + + + + true + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.13 + + build-resources/header-file.txt + build-resources/checkstyle.xml + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + + jar-no-fork + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.1 + + ${java.version} + + + + attach-javadocs + + jar + + + + + + org.apache.maven.plugins + maven-clean-plugin + 2.6.1 + + + org.apache.maven.plugins + maven-deploy-plugin + 2.8.2 + + + org.apache.maven.plugins + maven-install-plugin + 2.5.2 + + + org.apache.maven.plugins + maven-resources-plugin + 2.7 + + + org.apache.maven.plugins + maven-jar-plugin + 2.5 + + + org.apache.maven.plugins + maven-site-plugin + 3.4 + + + - maven-compiler-plugin - 3.1 - - 1.7 - 1.7 - - -Xlint:all,-serial - - + org.apache.maven.plugins + maven-checkstyle-plugin org.apache.maven.plugins - maven-checkstyle-plugin - 2.13 - - build-resources/header-file.txt - build-resources/checkstyle.xml - + maven-source-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin @@ -53,7 +143,7 @@ License. org.apache.spark spark-core_2.10 - 1.1.0 + 1.2.0 com.google.guava @@ -70,7 +160,7 @@ License. junit junit - 4.11 + 4.12 test @@ -80,9 +170,52 @@ License. org.apache.maven.plugins maven-checkstyle-plugin - 2.13 - + + + org.apache.maven.plugins + maven-javadoc-plugin + + + aggregate + + aggregate + test-aggregate + + + + + + http://github.com/cloudera/spark-dataflow + 2014 + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + Cloudera, Inc. + + + + + GitHub + https://github.com/cloudera/spark-dataflow/issues + + + scm:git:https://github.com/cloudera/spark-dataflow.git + scm:git:https://github.com/cloudera/spark-dataflow.git + scm:git:https://github.com/cloudera/spark-dataflow.git + HEAD + + + + 3.2.1 + + 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 index b6936e51cbb1..2ae917a0f319 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -18,10 +18,10 @@ 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.Lists; import org.apache.spark.api.java.function.Function; /** @@ -56,7 +56,7 @@ static byte[] toByteArray(T value, Coder coder) { * @return List of bytes representing serialized objects. */ static List toByteArrays(Iterable values, Coder coder) { - List res = Lists.newLinkedList(); + List res = new LinkedList<>(); for (T value : values) { res.add(toByteArray(value, coder)); } 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 index 673237e574e8..ec23bd29c475 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -52,10 +52,9 @@ class DoFnFunction implements FlatMapFunction, O> { * @param runtime Runtime to apply function in. * @param sideInputs Side inputs used in DoFunction. */ - public DoFnFunction( - DoFn fn, - SparkRuntimeContext runtime, - Map, BroadcastHelper> sideInputs) { + DoFnFunction(DoFn fn, + SparkRuntimeContext runtime, + Map, BroadcastHelper> sideInputs) { this.mFunction = fn; this.mRuntimeContext = runtime; this.mSideInputs = sideInputs; 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 index 4105ab61eb1f..09634111ec04 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -15,6 +15,8 @@ package com.cloudera.dataflow.spark; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,8 +34,6 @@ import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -45,9 +45,9 @@ public class EvaluationContext implements EvaluationResult { private final JavaSparkContext jsc; private final Pipeline pipeline; private final SparkRuntimeContext runtime; - private final Map> rdds = Maps.newHashMap(); - private final Set multireads = Sets.newHashSet(); - private final Map, Object> pobjects = Maps.newHashMap(); + private final Map> rdds = new HashMap<>(); + private final Set multireads = new HashSet<>(); + private final Map, Object> pobjects = new HashMap<>(); public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; 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 index 0c223e74eacc..b6911b5f0d1a 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -20,8 +20,8 @@ import com.google.cloud.dataflow.sdk.values.PObject; /** - * Interfacing for retrieve the result(s) of running a pipeline. Allows us to translate between - * PObjects or PCollections and T's or coolections of T's. + * Interface for retrieving the result(s) of running a pipeline. Allows us to translate between + * {@code PObject}s or {@code PCollection}s and Ts or collections of Ts. */ public interface EvaluationResult extends PipelineResult { /** @@ -45,8 +45,8 @@ public interface EvaluationResult extends PipelineResult { /** * Retrieves the final value of the aggregator. * - * @param aggName name of aggragtor. - * @param resultType Class of final result of aggregatiohn. + * @param aggName name of aggregator. + * @param resultType Class of final result of aggregation. * @param Type of final result of aggregation. * @return Result of aggregation associated with specified name. */ 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 index f8b261a125ee..ed12545028ce 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -29,12 +29,12 @@ * 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: - * Pipeline p = + * 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: - * Pipeline p = + * Pipeline p = [logic for pipeline creation] * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); * options.setSparkMaster("spark://host:port"); * EvaluationResult result = SparkPipelineRunner.create(options).run(p); 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 index 5b1e085d9a67..210b00fc2356 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -44,7 +44,7 @@ class SparkRuntimeContext implements Serializable { */ private final Map> aggregators = new HashMap<>(); - public SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { + SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); } @@ -122,7 +122,7 @@ public synchronized Aggregator createAggregator( private static class SparkAggregator implements Aggregator { private final NamedAggregators.State state; - public SparkAggregator(NamedAggregators.State state) { + SparkAggregator(NamedAggregators.State state) { this.state = state; } 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 index 06ee4d37c06e..7127b07d6f3d 100644 --- 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 @@ -99,7 +99,7 @@ public String toString() { /** * @param Input data type * @param Intermediate data type (useful for averages) - * @param Output datatype + * @param Output data type */ public interface State extends Serializable { /** @@ -115,11 +115,7 @@ public interface State extends Serializable { } /** - * => combineFunction in data flow. - * - * @param - * @param - * @param + * => combineFunction in data flow. */ public static class CombineFunctionState implements State { @@ -154,10 +150,7 @@ public Out render() { } /** - * states correspond to dataflow objects. this one => serializable function - * - * @param - * @param + * states correspond to dataflow objects. this one => serializable function */ public static class SerFunctionState implements State { 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 index ac884dd4c4ef..e87ee82ca540 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -76,11 +76,11 @@ public void testRun() throws Exception { */ static class ExtractWordsFn extends DoFn { - Aggregator totalWords; - Aggregator maxWordLength; - final PCollectionView regex; + private Aggregator totalWords; + private Aggregator maxWordLength; + private final PCollectionView regex; - public ExtractWordsFn(PCollectionView regex) { + ExtractWordsFn(PCollectionView regex) { this.regex = regex; } 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 index 7c6fd68dd85d..278f42c0e7f0 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -27,7 +27,7 @@ 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.Sets; +import com.google.common.collect.ImmutableSet; import org.junit.Assert; import org.junit.Test; @@ -41,8 +41,8 @@ public class SimpleWordCountTest { "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"}; private static final List WORDS = Arrays.asList(WORDS_ARRAY); - private static final Set EXPECTED_COUNT_SET = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", - "bob: 2"); + private static final Set EXPECTED_COUNT_SET = + ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); @Test public void testRun() throws Exception { From 1f9cd04591497231a65c83d6d3f4a27428be79ac Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 23 Dec 2014 18:55:18 -0800 Subject: [PATCH 029/137] Improve readme to explain current state of the repo, and to encourage outside contributions. --- runners/spark/README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/runners/spark/README.md b/runners/spark/README.md index 11ff192ffa14..f248bc28b7dc 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -1,5 +1,10 @@ spark-dataflow ============== + +Spark-dataflow is an early prototype. If this project interests you, you should know that we +encourage outside contributions. So, hack away! To get an idea of what we have already identified as +areas that need improvement, checkout the issues listed in the github repo. + Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing a pipeline on a spark cluster is easy: Depend on spark-dataflow in your project and execute your pipeline in a program by calling `SparkPipelineRunner.run`. From ba4b32687c1c1bbf86bec6fc3a87e9c90c078ad7 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Tue, 23 Dec 2014 10:44:46 -0800 Subject: [PATCH 030/137] Update version of dataflow we depend on. The primary change needed to accomodate the new dataflow api is to how we handle side inputs. --- runners/spark/pom.xml | 4 +- .../dataflow/spark/BroadcastHelper.java | 54 +- .../cloudera/dataflow/spark/CoderHelpers.java | 150 ++--- .../cloudera/dataflow/spark/DoFnFunction.java | 216 +++---- .../dataflow/spark/EvaluationContext.java | 275 ++++----- .../dataflow/spark/EvaluationResult.java | 52 +- .../dataflow/spark/MultiDoFnFunction.java | 204 +++--- .../dataflow/spark/SparkPipelineOptions.java | 1 + .../spark/SparkPipelineOptionsFactory.java | 2 +- .../dataflow/spark/SparkPipelineRunner.java | 30 +- .../dataflow/spark/SparkRuntimeContext.java | 170 ++--- .../dataflow/spark/TransformEvaluator.java | 2 +- .../dataflow/spark/TransformTranslator.java | 583 +++++++++--------- .../spark/aggregators/AggAccumParam.java | 24 +- .../spark/aggregators/NamedAggregators.java | 17 +- .../spark/MultiOutputWordCountTest.java | 14 +- .../dataflow/spark/SimpleWordCountTest.java | 7 +- 17 files changed, 916 insertions(+), 889 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index aece8908b55c..cab1b9a95089 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -152,8 +152,8 @@ License. com.google.cloud.dataflow - dataflow-sdk - 1.0.141206 + google-cloud-dataflow-java-sdk-all + 0.3.141216 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 index 8c728add5932..edad6692c9c2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -15,35 +15,47 @@ package com.cloudera.dataflow.spark; +import com.google.cloud.dataflow.sdk.coders.Coder; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; + import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.Serializable; - -import com.google.cloud.dataflow.sdk.coders.Coder; -import org.apache.spark.broadcast.Broadcast; +import java.util.logging.Logger; class BroadcastHelper implements Serializable { - private final Broadcast bcast; - private final Coder coder; - private transient T value; + private static Logger LOG = Logger.getLogger(BroadcastHelper.class.getName()); + private Broadcast bcast; + private final Coder coder; + private final T input; + private transient T value; - BroadcastHelper(Broadcast bcast, Coder coder) { - this.bcast = bcast; - this.coder = coder; - } + BroadcastHelper(T input, Coder coder) { + this.input = input; + this.coder = coder; + } - public synchronized T getValue() { - if (value == null) { - value = deserialize(); - } - return value; + public synchronized T getValue() { + if (value == null) { + value = deserialize(); } + return value; + } + + public void broadcast(JavaSparkContext jsc) { + this.bcast = jsc.broadcast(CoderHelpers.toByteArray(input, coder)); + } - private T deserialize() { - try { - return coder.decode(new ByteArrayInputStream(bcast.value()), new Coder.Context(true)); - } catch (IOException e) { - throw new IllegalStateException("Error deserializing broadcast variable", e); - } + 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.warning(ioe.getMessage()); + val = null; } + return val; + } } 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 index 2ae917a0f319..0a2dfaefd42d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -28,87 +28,87 @@ * Serialization utility class. */ public final class CoderHelpers { - private CoderHelpers() { - } + private CoderHelpers() { + } - /** - * Utility method for serializing an object using the specified coder. - * - * @param value Value to serialize. - * @param coder Coder to serialize with. - * @return Byte array representing serialized object. - */ - static byte[] toByteArray(T value, Coder 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 an object using the specified coder. + * + * @param value Value to serialize. + * @param coder Coder to serialize with. + * @return Byte array representing serialized object. + */ + static byte[] toByteArray(T value, Coder 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. - * @return List of bytes representing serialized objects. - */ - static List toByteArrays(Iterable values, Coder coder) { - List res = new LinkedList<>(); - for (T value : values) { - res.add(toByteArray(value, coder)); - } - return res; + /** + * Utility method for serializing a Iterable of values using the specified coder. + * + * @param values Values to serialize. + * @param coder Coder to serialize with. + * @return List of bytes representing serialized objects. + */ + static List toByteArrays(Iterable values, Coder coder) { + List 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 Type of object to be returned. - * @return Deserialized object. - */ - static T fromByteArray(byte[] serialized, Coder 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); - } + /** + * Utility method for deserializing a byte array using the specified coder. + * + * @param serialized bytearray to be deserialized. + * @param coder Coder to deserialize with. + * @param Type of object to be returned. + * @return Deserialized object. + */ + static T fromByteArray(byte[] serialized, Coder 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 The type of the object being serialized. - * @return A function that accepts an object and returns its coder-serialized form. - */ - static Function toByteFunction(final Coder coder) { - return new Function() { - @Override - public byte[] call(T t) throws Exception { - return toByteArray(t, coder); - } - }; - } + /** + * A function wrapper for converting an object to a bytearray. + * + * @param coder Coder to serialize with. + * @param The type of the object being serialized. + * @return A function that accepts an object and returns its coder-serialized form. + */ + static Function toByteFunction(final Coder coder) { + return new Function() { + @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 The type of the object being deserialized. - * @return A function that accepts a byte array and returns its corresponding object. - */ - static Function fromByteFunction(final Coder coder) { - return new Function() { - @Override - public T call(byte[] bytes) throws Exception { - return fromByteArray(bytes, coder); - } - }; - } + /** + * A function wrapper for converting a byte array to an object. + * + * @param coder Coder to deserialize with. + * @param The type of the object being deserialized. + * @return A function that accepts a byte array and returns its corresponding object. + */ + static Function fromByteFunction(final Coder coder) { + return new Function() { + @Override + public T call(byte[] bytes) throws Exception { + return fromByteArray(bytes, coder); + } + }; + } } 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 index ec23bd29c475..85a9cab92c94 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -15,13 +15,6 @@ package com.cloudera.dataflow.spark; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.logging.Logger; - import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -34,6 +27,13 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.logging.Logger; + /** * Dataflow's Do functions correspond to Spark's FlatMap functions. * @@ -41,112 +41,112 @@ * @param Output element type. */ class DoFnFunction implements FlatMapFunction, O> { - private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); - - private final DoFn mFunction; - private final SparkRuntimeContext mRuntimeContext; - private final Map, BroadcastHelper> mSideInputs; - - /** - * @param fn DoFunction to be wrapped. - * @param runtime Runtime to apply function in. - * @param sideInputs Side inputs used in DoFunction. - */ - DoFnFunction(DoFn fn, - SparkRuntimeContext runtime, - Map, BroadcastHelper> sideInputs) { - this.mFunction = fn; - this.mRuntimeContext = runtime; - this.mSideInputs = sideInputs; + private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); + + private final DoFn mFunction; + private final SparkRuntimeContext mRuntimeContext; + private final Map, BroadcastHelper> mSideInputs; + + /** + * @param fn DoFunction to be wrapped. + * @param runtime Runtime to apply function in. + * @param sideInputs Side inputs used in DoFunction. + */ + DoFnFunction(DoFn fn, + SparkRuntimeContext runtime, + Map, BroadcastHelper> sideInputs) { + this.mFunction = fn; + this.mRuntimeContext = runtime; + this.mSideInputs = sideInputs; + } + + + @Override + public Iterable call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt(mFunction); + //setup + mFunction.startBundle(ctxt); + //operation + while (iter.hasNext()) { + ctxt.element = iter.next(); + mFunction.processElement(ctxt); + } + //cleanup + mFunction.finishBundle(ctxt); + return ctxt.outputs; + } + + private class ProcCtxt extends DoFn.ProcessContext { + + private final List outputs = new LinkedList<>(); + private I element; + + public ProcCtxt(DoFn fn) { + fn.super(); } + @Override + public PipelineOptions getPipelineOptions() { + return mRuntimeContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + @SuppressWarnings("unchecked") + T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); + return value; + } @Override - public Iterable call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(mFunction); - //setup - mFunction.startBundle(ctxt); - //operation - while (iter.hasNext()) { - ctxt.element = iter.next(); - mFunction.processElement(ctxt); - } - //cleanup - mFunction.finishBundle(ctxt); - return ctxt.outputs; + public synchronized void output(O o) { + outputs.add(o); } - private class ProcCtxt extends DoFn.ProcessContext { - - private final List outputs = new LinkedList<>(); - private I element; - - public ProcCtxt(DoFn fn) { - fn.super(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return mRuntimeContext.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - @SuppressWarnings("unchecked") - T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); - return value; - } - - @Override - public synchronized void output(O o) { - outputs.add(o); - } - - @Override - public void sideOutput(TupleTag tupleTag, T t) { - LOG.warning("sideoutput is an unsupported operation for DoFnFunctions. Use a " + - "MultiDoFunction"); - throw new UnsupportedOperationException("sideOutput is an unsupported operation for " + - "doFunctions, use a MultiDoFunction instead."); - } - - @Override - public Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - return mRuntimeContext.createAggregator(named, combineFn); - } - - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return mRuntimeContext.createAggregator(named, sfunc); - } - - @Override - public I element() { - return element; - } - - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public Collection windows() { - return ImmutableList.of(); - } + @Override + public void sideOutput(TupleTag tupleTag, T t) { + String message = "sideOutput is an unsupported operation for doFunctions, use a" + + "MultiDoFunction instead."; + LOG.warning(message); + throw new UnsupportedOperationException(message); + } + + @Override + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return mRuntimeContext.createAggregator(named, combineFn); + } + + @Override + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return mRuntimeContext.createAggregator(named, sfunc); + } + + @Override + public I element() { + return element; + } + + @Override + public DoFn.KeyedState keyedState() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + output(output); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Collection windows() { + return ImmutableList.of(); } + } } 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 index 09634111ec04..6d51a8b12a15 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -15,162 +15,153 @@ package com.cloudera.dataflow.spark; -import java.util.HashMap; -import java.util.HashSet; -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.coders.CoderRegistry; 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.PObject; -import com.google.cloud.dataflow.sdk.values.PObjectTuple; -import com.google.cloud.dataflow.sdk.values.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; -import com.google.cloud.dataflow.sdk.values.TupleTag; 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; -import org.apache.spark.broadcast.Broadcast; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * 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> rdds = new HashMap<>(); - private final Set multireads = new HashSet<>(); - private final Map, Object> pobjects = new HashMap<>(); - - public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { - this.jsc = jsc; - this.pipeline = pipeline; - this.runtime = new SparkRuntimeContext(jsc, pipeline); - } - - JavaSparkContext getSparkContext() { - return jsc; - } - - Pipeline getPipeline() { - return pipeline; - } - - SparkRuntimeContext getRuntimeContext() { - return runtime; - } - - I getInput(PTransform transform) { - @SuppressWarnings("unchecked") - I input = (I) pipeline.getInput(transform); - return input; - } - - O getOutput(PTransform transform) { - @SuppressWarnings("unchecked") - O output = (O) pipeline.getOutput(transform); - return output; - } - - void setOutputRDD(PTransform transform, JavaRDDLike rdd) { - rdds.put((PValue) getOutput(transform), rdd); - } - - void setPObjectValue(PObject pobject, Object value) { - pobjects.put(pobject, value); - } - - JavaRDDLike getRDD(PValue pvalue) { - JavaRDDLike rdd = rdds.get(pvalue); - if (multireads.contains(pvalue)) { - // Ensure the RDD is marked as cached - rdd.rdd().cache(); - } else { - multireads.add(pvalue); - } - return rdd; - } - - void setRDD(PValue pvalue, JavaRDDLike rdd) { - rdds.put(pvalue, rdd); - } - - JavaRDDLike getInputRDD(PTransform transform) { - return getRDD((PValue) pipeline.getInput(transform)); - } - - BroadcastHelper getBroadcastHelper(PObject value) { - Coder coder = value.getCoder(); - Broadcast bcast = jsc.broadcast(CoderHelpers.toByteArray(get(value), coder)); - return new BroadcastHelper<>(bcast, coder); - } - - @Override - public T get(PObject value) { - if (pobjects.containsKey(value)) { - @SuppressWarnings("unchecked") - T result = (T) pobjects.get(value); - return result; - } - if (rdds.containsKey(value)) { - JavaRDDLike rdd = rdds.get(value); - //TODO: need same logic from get() method below here for serialization of bytes - @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 getAggregatorValue(String named, Class resultType) { - return runtime.getAggregatorValue(named, resultType); - } - - @Override - public Iterable get(PCollection pcollection) { - @SuppressWarnings("unchecked") - JavaRDDLike rdd = (JavaRDDLike) getRDD(pcollection); - final Coder coder = pcollection.getCoder(); - JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); - List clientBytes = bytesRDD.collect(); - return Iterables.transform(clientBytes, new Function() { - @Override - public T apply(byte[] bytes) { - return (T) CoderHelpers.fromByteArray(bytes, coder); - } - }); - } - - PObjectValueTuple getPObjectTuple(PTransform transform) { - PObjectTuple pot = (PObjectTuple) pipeline.getInput(transform); - PObjectValueTuple povt = PObjectValueTuple.empty(); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - povt = and(povt, e); - } - return povt; - } - - private PObjectValueTuple and(PObjectValueTuple povt, Map.Entry, PObject> e) { - @SuppressWarnings("unchecked") - TupleTag ttKey = (TupleTag) e.getKey(); - @SuppressWarnings("unchecked") - PObject potValue = (PObject) e.getValue(); - return povt.and(ttKey, get(potValue)); - } - - void setPObjectTuple(PTransform transform, PObjectValueTuple outputValues) { - PObjectTuple pot = (PObjectTuple) pipeline.getOutput(transform); - for (Map.Entry, PObject> e : pot.getAll().entrySet()) { - pobjects.put(e.getValue(), outputValues.get(e.getKey())); - } - } + private final JavaSparkContext jsc; + private final Pipeline pipeline; + private final SparkRuntimeContext runtime; + private final CoderRegistry registry; + private final Map> rdds = new HashMap<>(); + private final Set multireads = new HashSet<>(); + private final Map pobjects = new HashMap<>(); + private final Map>> pview = new HashMap<>(); + + public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { + this.jsc = jsc; + this.pipeline = pipeline; + this.registry = pipeline.getCoderRegistry(); + this.runtime = new SparkRuntimeContext(jsc, pipeline); + } + + JavaSparkContext getSparkContext() { + return jsc; + } + + Pipeline getPipeline() { + return pipeline; + } + + SparkRuntimeContext getRuntimeContext() { + return runtime; + } + + Coder getDefaultCoder(T example) { + Coder defaultCoder = registry.getDefaultCoder(example); + if (defaultCoder == null) { + throw new IllegalStateException(String.format("Couldn't determine the default coder for " + + "an example of class [%s]", example.getClass())); + } + return defaultCoder; + } + + /** + * Coder> getDefaultIterableCoder(Iterables iter) { + *

+ * } + */ + + I getInput(PTransform transform) { + @SuppressWarnings("unchecked") + I input = (I) pipeline.getInput(transform); + return input; + } + + O getOutput(PTransform transform) { + @SuppressWarnings("unchecked") + O output = (O) pipeline.getOutput(transform); + return output; + } + + void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + rdds.put((PValue) getOutput(transform), rdd); + } + + void setPView(PValue view, Iterable> value) { + pview.put(view, value); + } + + JavaRDDLike getRDD(PValue pvalue) { + JavaRDDLike rdd = rdds.get(pvalue); + if (multireads.contains(pvalue)) { + // Ensure the RDD is marked as cached + rdd.rdd().cache(); + } else { + multireads.add(pvalue); + } + return rdd; + } + + void setRDD(PValue pvalue, JavaRDDLike rdd) { + rdds.put(pvalue, rdd); + } + + JavaRDDLike getInputRDD(PTransform transform) { + return getRDD((PValue) pipeline.getInput(transform)); + } + + + Iterable> getPCollectionView(PCollectionView view) { + Iterable> value = pview.get(view); + return value; + } + + @Override + public T get(PValue value) { + if (pobjects.containsKey(value)) { + @SuppressWarnings("unchecked") + T result = (T) pobjects.get(value); + return result; + } + if (rdds.containsKey(value)) { + JavaRDDLike rdd = rdds.get(value); + @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 getAggregatorValue(String named, Class resultType) { + return runtime.getAggregatorValue(named, resultType); + } + + @Override + public Iterable get(PCollection pcollection) { + @SuppressWarnings("unchecked") + JavaRDDLike rdd = (JavaRDDLike) getRDD(pcollection); + final Coder coder = pcollection.getCoder(); + JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); + List clientBytes = bytesRDD.collect(); + return Iterables.transform(clientBytes, new Function() { + @Override + public T apply(byte[] bytes) { + return (T) CoderHelpers.fromByteArray(bytes, coder); + } + }); + } } 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 index b6911b5f0d1a..871a0c100f95 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -17,38 +17,38 @@ import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.cloud.dataflow.sdk.values.PObject; +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}s or {@code PCollection}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 Type of elements contained in collection. - * @return Natively types result associated with collection. - */ - Iterable get(PCollection pcollection); + /** + * Retrieves an iterable of results associated with the PCollection passed in. + * + * @param pcollection Collection we wish to translate. + * @param Type of elements contained in collection. + * @return Natively types result associated with collection. + */ + Iterable get(PCollection pcollection); - /** - * Retrieve an object of Type T associated with the PObject passed in. - * - * @param pobject PObject we wish to translate. - * @param Type of object to return. - * @return Native object. - */ - T get(PObject pobject); + /** + * Retrieve an object of Type T associated with the PValue passed in. + * + * @param pval PValue to retireve associated data for. + * @param Type of object to return. + * @return Native object. + */ + 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 Type of final result of aggregation. - * @return Result of aggregation associated with specified name. - */ - T getAggregatorValue(String aggName, Class resultType); + /** + * Retrieves the final value of the aggregator. + * + * @param aggName name of aggregator. + * @param resultType Class of final result of aggregation. + * @param Type of final result of aggregation. + * @return Result of aggregation associated with specified name. + */ + T getAggregatorValue(String aggName, Class resultType); } 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 index 70facf82bb4d..60447f89f83f 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -44,112 +44,112 @@ * @param Output type for DoFunction. */ class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { - // TODO: I think implementing decoding logic will allow us to do away with having two types of - // DoFunctions. Josh originally made these two classes in order to help ease the typing of - // results. Correctly using coders should just fix this. - - private final DoFn mFunction; - private final SparkRuntimeContext mRuntimeContext; - private final TupleTag mMainOutputTag; - private final Map, BroadcastHelper> mSideInputs; - - MultiDoFnFunction( - DoFn fn, - SparkRuntimeContext runtimeContext, - TupleTag mainOutputTag, - Map, BroadcastHelper> sideInputs) { - this.mFunction = fn; - this.mRuntimeContext = runtimeContext; - this.mMainOutputTag = mainOutputTag; - this.mSideInputs = sideInputs; + // TODO: I think implementing decoding logic will allow us to do away with having two types of + // DoFunctions. Josh originally made these two classes in order to help ease the typing of + // results. Correctly using coders should just fix this. + + private final DoFn mFunction; + private final SparkRuntimeContext mRuntimeContext; + private final TupleTag mMainOutputTag; + private final Map, BroadcastHelper> mSideInputs; + + MultiDoFnFunction( + DoFn fn, + SparkRuntimeContext runtimeContext, + TupleTag mainOutputTag, + Map, BroadcastHelper> sideInputs) { + this.mFunction = fn; + this.mRuntimeContext = runtimeContext; + this.mMainOutputTag = mainOutputTag; + this.mSideInputs = sideInputs; + } + + @Override + public Iterable, Object>> call(Iterator iter) throws Exception { + ProcCtxt ctxt = new ProcCtxt(mFunction); + mFunction.startBundle(ctxt); + while (iter.hasNext()) { + ctxt.element = iter.next(); + mFunction.processElement(ctxt); + } + mFunction.finishBundle(ctxt); + return Iterables.transform(ctxt.outputs.entries(), + new Function, Object>, Tuple2, Object>>() { + @Override + public Tuple2, Object> apply(Map.Entry, Object> input) { + return new Tuple2, Object>(input.getKey(), input.getValue()); + } + }); + } + + private class ProcCtxt extends DoFn.ProcessContext { + + private final Multimap, Object> outputs = LinkedListMultimap.create(); + private I element; + + ProcCtxt(DoFn fn) { + fn.super(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return mRuntimeContext.getPipelineOptions(); } @Override - public Iterable, Object>> call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(mFunction); - mFunction.startBundle(ctxt); - while (iter.hasNext()) { - ctxt.element = iter.next(); - mFunction.processElement(ctxt); - } - mFunction.finishBundle(ctxt); - return Iterables.transform(ctxt.outputs.entries(), - new Function, Object>, Tuple2, Object>>() { - @Override - public Tuple2, Object> apply(Map.Entry, Object> input) { - return new Tuple2, Object>(input.getKey(), input.getValue()); - } - }); + public T sideInput(PCollectionView view) { + @SuppressWarnings("unchecked") + T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); + return value; } - private class ProcCtxt extends DoFn.ProcessContext { - - private final Multimap, Object> outputs = LinkedListMultimap.create(); - private I element; - - ProcCtxt(DoFn fn) { - fn.super(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return mRuntimeContext.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - @SuppressWarnings("unchecked") - T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); - return value; - } - - @Override - public synchronized void output(O o) { - outputs.put(mMainOutputTag, o); - } - - @Override - public synchronized void sideOutput(TupleTag tag, T t) { - outputs.put(tag, t); - } - - @Override - public Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - return mRuntimeContext.createAggregator(named, combineFn); - } - - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return mRuntimeContext.createAggregator(named, sfunc); - } - - @Override - public I element() { - return element; - } - - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public Collection windows() { - return ImmutableList.of(); - } + @Override + public synchronized void output(O o) { + outputs.put(mMainOutputTag, o); + } + + @Override + public synchronized void sideOutput(TupleTag tag, T t) { + outputs.put(tag, t); + } + + @Override + public Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + return mRuntimeContext.createAggregator(named, combineFn); + } + + @Override + public Aggregator createAggregator( + String named, + SerializableFunction, AO> sfunc) { + return mRuntimeContext.createAggregator(named, sfunc); + } + + @Override + public I element() { + return element; + } + + @Override + public DoFn.KeyedState keyedState() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + output(output); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Collection windows() { + return ImmutableList.of(); } + } } 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 index d5d96afcbb0d..06793065649c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java @@ -23,5 +23,6 @@ public interface SparkPipelineOptions extends PipelineOptions { @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); } 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 index 510038cc13ae..89cd0302b522 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsFactory.java @@ -22,6 +22,6 @@ private SparkPipelineOptionsFactory() { } public static SparkPipelineOptions create() { - return PipelineOptionsFactory.create(SparkPipelineOptions.class); + return PipelineOptionsFactory.as(SparkPipelineOptions.class); } } 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 index ed12545028ce..b0a7c4513ba3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -20,6 +20,7 @@ import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.values.PValue; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import java.util.logging.Logger; @@ -29,21 +30,22 @@ * 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: - * 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: - * Pipeline p = [logic for pipeline creation] - * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); - * options.setSparkMaster("spark://host:port"); - * EvaluationResult result = SparkPipelineRunner.create(options).run(p); - * + * 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: + * Pipeline p = [logic for pipeline creation] + * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + * options.setSparkMaster("spark://host:port"); + * EvaluationResult result = SparkPipelineRunner.create(options).run(p); */ public class SparkPipelineRunner extends PipelineRunner { private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); - /** Options used in this pipeline runner.*/ + /** + * Options used in this pipeline runner. + */ private final SparkPipelineOptions mOptions; /** @@ -86,7 +88,11 @@ public EvaluationResult run(Pipeline pipeline) { } private JavaSparkContext getContext() { - return new JavaSparkContext(mOptions.getSparkMaster(), mOptions.getJobName()); + SparkConf conf = new SparkConf(); + conf.setMaster(mOptions.getSparkMaster()); + conf.setAppName("spark pipeline job"); + conf.set("sun.io.serialization.extendeddebuginfo", "true"); + return new JavaSparkContext(conf); } private static class Evaluator implements Pipeline.PipelineVisitor { 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 index 210b00fc2356..198f0d65aedb 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -35,100 +35,100 @@ * data flow program is launched. */ class SparkRuntimeContext implements Serializable { - /** - * An accumulator that is a map from names to aggregators. - */ - private final Accumulator accum; - /** - * Map fo names to dataflow aggregators. - */ - private final Map> aggregators = new HashMap<>(); + /** + * An accumulator that is a map from names to aggregators. + */ + private final Accumulator accum; + /** + * Map fo names to dataflow aggregators. + */ + private final Map> aggregators = new HashMap<>(); - SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); - } + SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { + this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); + } - /** - * 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 Type of object to be returned. - * @return The value of the aggregator. - */ - public T getAggregatorValue(String aggregatorName, Class typeClass) { - return accum.value().getValue(aggregatorName, typeClass); - } + /** + * 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 Type of object to be returned. + * @return The value of the aggregator. + */ + public T getAggregatorValue(String aggregatorName, Class typeClass) { + return accum.value().getValue(aggregatorName, typeClass); + } - public synchronized PipelineOptions getPipelineOptions() { - //TODO: Support this. - throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); - } + public synchronized PipelineOptions getPipelineOptions() { + //TODO: Support this. + throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); + } - /** - * Creates and aggregator and associates it with the specified name. - * - * @param named Name of aggregator. - * @param sfunc Serializable function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. - * @return Specified aggregator - */ - public synchronized Aggregator createAggregator( - String named, - SerializableFunction, Out> sfunc) { - @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); - if (aggregator == null) { - NamedAggregators.SerFunctionState state = new NamedAggregators - .SerFunctionState<>(sfunc); - accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator<>(state); - aggregators.put(named, aggregator); - } - return aggregator; + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param sfunc Serializable function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( + String named, + SerializableFunction, Out> sfunc) { + @SuppressWarnings("unchecked") + Aggregator aggregator = (Aggregator) aggregators.get(named); + if (aggregator == null) { + NamedAggregators.SerFunctionState state = new NamedAggregators + .SerFunctionState<>(sfunc); + accum.add(new NamedAggregators(named, state)); + aggregator = new SparkAggregator<>(state); + aggregators.put(named, aggregator); } + return aggregator; + } - /** - * Creates and aggregator and associates it with the specified name. - * - * @param named Name of aggregator. - * @param combineFn Combine function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. - * @return Specified aggregator - */ - public synchronized Aggregator createAggregator( - String named, - Combine.CombineFn combineFn) { - @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); - if (aggregator == null) { - @SuppressWarnings("unchecked") - NamedAggregators.CombineFunctionState state = new NamedAggregators - .CombineFunctionState<>((Combine.CombineFn) combineFn); - accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator<>(state); - aggregators.put(named, aggregator); - } - return aggregator; + /** + * Creates and aggregator and associates it with the specified name. + * + * @param named Name of aggregator. + * @param combineFn Combine function used in aggregation. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. + * @return Specified aggregator + */ + public synchronized Aggregator createAggregator( + String named, + Combine.CombineFn combineFn) { + @SuppressWarnings("unchecked") + Aggregator aggregator = (Aggregator) aggregators.get(named); + if (aggregator == null) { + @SuppressWarnings("unchecked") + NamedAggregators.CombineFunctionState state = new NamedAggregators + .CombineFunctionState<>((Combine.CombineFn) combineFn); + accum.add(new NamedAggregators(named, state)); + aggregator = new SparkAggregator<>(state); + aggregators.put(named, aggregator); } + return aggregator; + } - /** - * Initialize spark aggregators exactly once. - * - * @param Type of element fed in to aggregator. - */ - private static class SparkAggregator implements Aggregator { - private final NamedAggregators.State state; + /** + * Initialize spark aggregators exactly once. + * + * @param Type of element fed in to aggregator. + */ + private static class SparkAggregator implements Aggregator { + private final NamedAggregators.State state; - SparkAggregator(NamedAggregators.State state) { - this.state = state; - } + SparkAggregator(NamedAggregators.State state) { + this.state = state; + } - @Override - public void addValue(In elem) { - state.update(elem); - } + @Override + public void addValue(In elem) { + state.update(elem); } + } } 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 index 6173920681c7..bf45d129b966 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java @@ -20,5 +20,5 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; public interface TransformEvaluator extends Serializable { - void evaluate(PT transform, EvaluationContext context); + void evaluate(PT transform, EvaluationContext context); } 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 index 477a8441bebe..199480289479 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -20,21 +20,18 @@ 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.Convert; import com.google.cloud.dataflow.sdk.transforms.Create; -import com.google.cloud.dataflow.sdk.transforms.CreatePObject; 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.SeqDo; +import com.google.cloud.dataflow.sdk.transforms.View; 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.PObjectValueTuple; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -46,311 +43,325 @@ import scala.Tuple2; import java.lang.reflect.Field; +import java.util.List; import java.util.Map; +import java.util.logging.Logger; /** * Supports translation between a DataFlow transform, and Spark's operations on RDDs. */ public final class TransformTranslator { - private TransformTranslator() { - } - - private static class FieldGetter { - private final Map fields; + private static final Logger LOG = Logger.getLogger(TransformTranslator.class.getName()); - FieldGetter(Class clazz) { - this.fields = Maps.newHashMap(); - for (Field f : clazz.getDeclaredFields()) { - f.setAccessible(true); - this.fields.put(f.getName(), f); - } - } + private TransformTranslator() { + } - public 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 TransformEvaluator> flatten() { - return new TransformEvaluator>() { - @SuppressWarnings("unchecked") - @Override - public void evaluate(Flatten transform, EvaluationContext context) { - PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); - JavaRDD[] rdds = new JavaRDD[pcs.size()]; - for (int i = 0; i < rdds.length; i++) { - rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); - } - JavaRDD rdd = context.getSparkContext().union(rdds); - context.setOutputRDD(transform, rdd); - } - }; - } + private static class FieldGetter { + private final Map fields; - private static TransformEvaluator> gbk() { - return new TransformEvaluator>() { - @Override - public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { - @SuppressWarnings("unchecked") - JavaRDDLike,?> inRDD = - (JavaRDDLike,?>) context.getInputRDD(transform); - context.setOutputRDD(transform, fromPair(toPair(inRDD).groupByKey())); - } - }; + FieldGetter(Class clazz) { + this.fields = Maps.newHashMap(); + for (Field f : clazz.getDeclaredFields()) { + f.setAccessible(true); + this.fields.put(f.getName(), f); + } } - private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); - private static TransformEvaluator> grouped() { - return new TransformEvaluator>() { - @Override - public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); - @SuppressWarnings("unchecked") - JavaRDDLike>,?> inRDD = - (JavaRDDLike>,?>) context.getInputRDD(transform); - context.setOutputRDD(transform, inRDD.map(new KVFunction<>(keyed))); - } - }; + public 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 final class KVFunction implements Function>, KV> { - private final Combine.KeyedCombineFn keyed; - - KVFunction(Combine.KeyedCombineFn keyed) { - this.keyed = keyed; + } + + private static TransformEvaluator> flattenPColl() { + return new TransformEvaluator>() { + @SuppressWarnings("unchecked") + @Override + public void evaluate(Flatten.FlattenPCollectionList transform, EvaluationContext context) { + PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + JavaRDD[] rdds = new JavaRDD[pcs.size()]; + for (int i = 0; i < rdds.length; i++) { + rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); } - - @Override - public KV call(KV> kv) throws Exception { - return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); - } - } - - private static JavaPairRDD toPair(JavaRDDLike,?> rdd) { - return rdd.mapToPair(new PairFunction,K,V>() { - @Override - public Tuple2 call(KV kv) { - return new Tuple2<>(kv.getKey(), kv.getValue()); - } - }); - } - - private static JavaRDDLike,?> fromPair(JavaPairRDD rdd) { - return rdd.map(new Function,KV>() { - @Override - public KV call(Tuple2 t2) { - return KV.of(t2._1(), t2._2()); - } - }); - } - - private static TransformEvaluator> parDo() { - return new TransformEvaluator>() { - @Override - public void evaluate(ParDo.Bound transform, EvaluationContext context) { - DoFnFunction dofn = - new DoFnFunction<>(transform.getFn(), - context.getRuntimeContext(), - getSideInputs(transform.getSideInputs(), context)); - @SuppressWarnings("unchecked") - JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); - context.setOutputRDD(transform, inRDD.mapPartitions(dofn)); - } - }; - } - - private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); - private static TransformEvaluator> multiDo() { - return new TransformEvaluator>() { - @Override - public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - TupleTag mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform); - MultiDoFnFunction multifn = new MultiDoFnFunction<>( - transform.getFn(), - context.getRuntimeContext(), - mainOutputTag, - getSideInputs(transform.getSideInputs(), context)); - - @SuppressWarnings("unchecked") - JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); - JavaPairRDD, Object> all = inRDD - .mapPartitionsToPair(multifn) - .cache(); - - PCollectionTuple pct = context.getOutput(transform); - for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { - @SuppressWarnings("unchecked") - JavaPairRDD, Object> filtered = - all.filter(new TupleTagFilter(e.getKey())); - context.setRDD(e.getValue(), filtered.values()); - } - } - }; - } - - - private static TransformEvaluator> readText() { - return new TransformEvaluator>() { - @Override - public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; - } - - private static TransformEvaluator> writeText() { - return new TransformEvaluator>() { - @Override - public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { - @SuppressWarnings("unchecked") - JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; - } - - private static TransformEvaluator> readAvro() { - return new TransformEvaluator>() { - @Override - public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { - String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); - context.setOutputRDD(transform, rdd); - } - }; - } - - private static TransformEvaluator> writeAvro() { - return new TransformEvaluator>() { - @Override - public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - @SuppressWarnings("unchecked") - JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); - } - }; - } - - private static TransformEvaluator> create() { - return new TransformEvaluator>() { - @Override - public void evaluate(Create transform, EvaluationContext context) { - Iterable elems = transform.getElements(); - Coder coder = context.getOutput(transform).getCoder(); - JavaRDD rdd = context.getSparkContext().parallelize( - CoderHelpers.toByteArrays(elems, coder)); - context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); - } - }; - } - - private static TransformEvaluator> createPObj() { - return new TransformEvaluator>() { - @Override - public void evaluate(CreatePObject transform, EvaluationContext context) { - context.setPObjectValue(context.getOutput(transform), transform.getElement()); - } - }; - } - - private static TransformEvaluator> toIter() { - return new TransformEvaluator>() { - @Override - public void evaluate(Convert.ToIterable transform, EvaluationContext context) { - context.setPObjectValue(context.getOutput(transform), - context.get(context.getInput(transform))); - } - }; + JavaRDD rdd = context.getSparkContext().union(rdds); + context.setOutputRDD(transform, rdd); + } + }; + } + + private static TransformEvaluator> gbk() { + return new TransformEvaluator>() { + @Override + public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike, ?> inRDD = + (JavaRDDLike, ?>) context.getInputRDD(transform); + context.setOutputRDD(transform, fromPair(toPair(inRDD).groupByKey())); + } + }; + } + + private static final FieldGetter GROUPED_FG = new FieldGetter(Combine.GroupedValues.class); + + private static TransformEvaluator> grouped() { + return new TransformEvaluator>() { + @Override + public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { + Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + @SuppressWarnings("unchecked") + JavaRDDLike>, ?> inRDD = + (JavaRDDLike>, ?>) context.getInputRDD(transform); + context.setOutputRDD(transform, inRDD.map(new KVFunction<>(keyed))); + } + }; + } + + private static final class KVFunction implements Function>, KV> { + private final Combine.KeyedCombineFn keyed; + + KVFunction(Combine.KeyedCombineFn keyed) { + this.keyed = keyed; } - private static TransformEvaluator> toIterWin() { - return new TransformEvaluator>() { - @Override - public void evaluate(Convert.ToIterableWindowedValue transform, EvaluationContext context) { - context.setPObjectValue(context.getOutput(transform), - Iterables.transform(context.get(context.getInput(transform)), - new com.google.common.base.Function>() { - @Override - public WindowedValue apply(T t) { - return WindowedValue.valueInGlobalWindow(t); - } - })); - } - }; + @Override + public KV call(KV> kv) throws Exception { + return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); } + } + + private static JavaPairRDD toPair(JavaRDDLike, ?> rdd) { + return rdd.mapToPair(new PairFunction, K, V>() { + @Override + public Tuple2 call(KV kv) { + return new Tuple2<>(kv.getKey(), kv.getValue()); + } + }); + } + + private static JavaRDDLike, ?> fromPair(JavaPairRDD rdd) { + return rdd.map(new Function, KV>() { + @Override + public KV call(Tuple2 t2) { + return KV.of(t2._1(), t2._2()); + } + }); + } + + private static TransformEvaluator> parDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + DoFnFunction dofn = + new DoFnFunction<>(transform.getFn(), + context.getRuntimeContext(), + getSideInputs(transform.getSideInputs(), context)); + @SuppressWarnings("unchecked") + JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); + context.setOutputRDD(transform, inRDD.mapPartitions(dofn)); + } + }; + } + + private static final FieldGetter MULTIDO_FG = new FieldGetter(ParDo.BoundMulti.class); + + private static TransformEvaluator> multiDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + TupleTag mainOutputTag = MULTIDO_FG.get("mainOutputTag", transform); + MultiDoFnFunction multifn = new MultiDoFnFunction<>( + transform.getFn(), + context.getRuntimeContext(), + mainOutputTag, + getSideInputs(transform.getSideInputs(), context)); - private static class TupleTagFilter implements Function, Object>, Boolean> { - private final TupleTag tag; - - private TupleTagFilter(TupleTag tag) { - this.tag = tag; - } - - @Override - public Boolean call(Tuple2, Object> input) { - return tag.equals(input._1()); + @SuppressWarnings("unchecked") + JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); + JavaPairRDD, Object> all = inRDD + .mapPartitionsToPair(multifn) + .cache(); + + PCollectionTuple pct = context.getOutput(transform); + for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { + @SuppressWarnings("unchecked") + JavaPairRDD, Object> filtered = + all.filter(new TupleTagFilter(e.getKey())); + context.setRDD(e.getValue(), filtered.values()); } + } + }; + } + + + private static TransformEvaluator> readText() { + return new TransformEvaluator>() { + @Override + public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + } + + private static TransformEvaluator> writeText() { + return new TransformEvaluator>() { + @Override + public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + } + + private static TransformEvaluator> readAvro() { + return new TransformEvaluator>() { + @Override + public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaRDD rdd = context.getSparkContext().textFile(pattern); + context.setOutputRDD(transform, rdd); + } + }; + } + + private static TransformEvaluator> writeAvro() { + return new TransformEvaluator>() { + @Override + public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); + String pattern = transform.getFilenamePrefix(); + last.saveAsTextFile(pattern); + } + }; + } + + private static TransformEvaluator> create() { + return new TransformEvaluator>() { + @Override + public void evaluate(Create transform, EvaluationContext context) { + Iterable elems = transform.getElements(); + Coder coder = context.getOutput(transform).getCoder(); + JavaRDD rdd = context.getSparkContext().parallelize( + CoderHelpers.toByteArrays(elems, coder)); + context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + } + + private static TransformEvaluator> viewAsSingleton() { + return new TransformEvaluator>() { + @Override + public void evaluate(View.AsSingleton transform, EvaluationContext context) { + //PROBABLY INCORRECT. Fix it. + Iterable input = context.get(context.getInput(transform)); + context.setPView(context.getOutput(transform), Iterables.transform(input, + new WindowingFunction())); + } + }; + } + + private static TransformEvaluator> viewAsIter() { + return new TransformEvaluator>() { + @Override + public void evaluate(View.AsIterable transform, EvaluationContext context) { + Iterable input = context.get(context.getInput(transform)); + + context.setPView(context.getOutput(transform), Iterables.transform(input, + new WindowingFunction())); + } + }; + } + + private static TransformEvaluator> createPCollView() { + return new TransformEvaluator>() { + @Override + public void evaluate(View.CreatePCollectionView transform, EvaluationContext + context) { + Iterable> iter = Iterables.transform(context.get(context.getInput + (transform)), new WindowingFunction() + ); + + context.setPView(context.getOutput(transform), iter); + } + }; + } + + private static class WindowingFunction implements com.google.common.base.Function> { + @Override + public WindowedValue apply(R t) { + return WindowedValue.valueInGlobalWindow(t); } + } - private static TransformEvaluator seqDo() { - return new TransformEvaluator() { - @Override - public void evaluate(SeqDo.BoundMulti transform, EvaluationContext context) { - PObjectValueTuple inputValues = context.getPObjectTuple(transform); - PObjectValueTuple outputValues = transform.getFn().process(inputValues); - context.setPObjectTuple(transform, outputValues); - } - }; - } + private static class TupleTagFilter implements Function, Object>, Boolean> { + private final TupleTag tag; - private static Map, BroadcastHelper> getSideInputs( - Iterable> views, - EvaluationContext context) { - if (views == null) { - return ImmutableMap.of(); - } else { - Map, BroadcastHelper>sideInputs = Maps.newHashMap(); - for (PCollectionView view : views) { - sideInputs.put(view.getTagInternal(), context.getBroadcastHelper(view.getPObjectInternal())); - } - return sideInputs; - } + private TupleTagFilter(TupleTag tag) { + this.tag = tag; } - private static final Map, TransformEvaluator> mEvaluators = Maps.newHashMap(); - static { - mEvaluators.put(TextIO.Read.Bound.class, readText()); - mEvaluators.put(TextIO.Write.Bound.class, writeText()); - mEvaluators.put(AvroIO.Read.Bound.class, readAvro()); - mEvaluators.put(AvroIO.Write.Bound.class, writeAvro()); - mEvaluators.put(ParDo.Bound.class, parDo()); - mEvaluators.put(ParDo.BoundMulti.class, multiDo()); - mEvaluators.put(SeqDo.BoundMulti.class, seqDo()); - mEvaluators.put(GroupByKey.GroupByKeyOnly.class, gbk()); - mEvaluators.put(Combine.GroupedValues.class, grouped()); - mEvaluators.put(Flatten.class, flatten()); - mEvaluators.put(Create.class, create()); - mEvaluators.put(CreatePObject.class, createPObj()); - mEvaluators.put(Convert.ToIterable.class, toIter()); - mEvaluators.put(Convert.ToIterableWindowedValue.class, toIterWin()); + @Override + public Boolean call(Tuple2, Object> input) { + return tag.equals(input._1()); } - - public static TransformEvaluator getTransformEvaluator(Class clazz) { - @SuppressWarnings("unchecked") - TransformEvaluator transform = (TransformEvaluator) mEvaluators.get(clazz); - if (transform == null) { - throw new IllegalStateException("No TransformEvaluator registered for " + clazz); - } - return transform; + } + + private static Map, BroadcastHelper> getSideInputs( + List> views, + EvaluationContext context) { + if (views == null) { + return ImmutableMap.of(); + } else { + Map, BroadcastHelper> sideInputs = Maps.newHashMap(); + for (PCollectionView view : views) { + Object sideinput = view.fromIterableInternal(context.getPCollectionView(view)); + Coder coder = context.getDefaultCoder(sideinput); + BroadcastHelper helper = new BroadcastHelper<>(sideinput, coder); + //broadcast side inputs + helper.broadcast(context.getSparkContext()); + sideInputs.put(view.getTagInternal(), helper); + } + return sideInputs; + } + } + + private static final Map, TransformEvaluator> mEvaluators = Maps + .newHashMap(); + + static { + mEvaluators.put(TextIO.Read.Bound.class, readText()); + mEvaluators.put(TextIO.Write.Bound.class, writeText()); + mEvaluators.put(AvroIO.Read.Bound.class, readAvro()); + mEvaluators.put(AvroIO.Write.Bound.class, writeAvro()); + mEvaluators.put(ParDo.Bound.class, parDo()); + mEvaluators.put(ParDo.BoundMulti.class, multiDo()); + mEvaluators.put(GroupByKey.GroupByKeyOnly.class, gbk()); + mEvaluators.put(Combine.GroupedValues.class, grouped()); + mEvaluators.put(Flatten.FlattenPCollectionList.class, flattenPColl()); + mEvaluators.put(Create.class, create()); + mEvaluators.put(View.AsSingleton.class, viewAsSingleton()); + mEvaluators.put(View.AsIterable.class, viewAsIter()); + mEvaluators.put(View.CreatePCollectionView.class, createPCollView()); + } + + public static TransformEvaluator getTransformEvaluator(Class + clazz) { + @SuppressWarnings("unchecked") + TransformEvaluator transform = (TransformEvaluator) mEvaluators.get(clazz); + if (transform == null) { + throw new IllegalStateException("No TransformEvaluator registered for " + clazz); } + return transform; + } } 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 index 918c52c98b2e..a3055a256ded 100644 --- 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 @@ -18,18 +18,18 @@ import org.apache.spark.AccumulatorParam; public class AggAccumParam implements AccumulatorParam { - @Override - public NamedAggregators addAccumulator(NamedAggregators current, NamedAggregators added) { - return current.merge(added); - } + @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 addInPlace(NamedAggregators current, NamedAggregators added) { + return addAccumulator(current, added); + } - @Override - public NamedAggregators zero(NamedAggregators initialValue) { - return new NamedAggregators(); - } + @Override + public NamedAggregators zero(NamedAggregators initialValue) { + return new NamedAggregators(); + } } 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 index 7127b07d6f3d..fde44ff4c8d3 100644 --- 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 @@ -30,9 +30,14 @@ * is launched. We can then add aggregators on the fly in Spark. */ public class NamedAggregators implements Serializable { - /** Map from aggregator name to current state. */ + /** + * Map from aggregator name to current state. + */ private final Map> mNamedAggregators = new TreeMap<>(); - /** Constructs a new NamedAggregators instance. */ + + /** + * Constructs a new NamedAggregators instance. + */ public NamedAggregators() { } @@ -40,7 +45,7 @@ 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 name Name of aggregator. * @param state Associated State. */ public NamedAggregators(String name, State state) { @@ -48,9 +53,9 @@ public NamedAggregators(String name, State state) { } /** - * @param name Name of aggregator to retrieve. + * @param name Name of aggregator to retrieve. * @param typeClass Type class to cast the value to. - * @param Type to be returned. + * @param Type to be returned. * @return the value of the aggregator associated with the specified name */ public T getValue(String name, Class typeClass) { @@ -62,7 +67,7 @@ public T getValue(String name, Class typeClass) { * * @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. + * other instance's aggregators. */ public NamedAggregators merge(NamedAggregators other) { for (Map.Entry> e : other.mNamedAggregators.entrySet()) { 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 index e87ee82ca540..74edc1d29749 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -19,7 +19,6 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; 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; @@ -28,12 +27,12 @@ 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.SingletonPCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.cloud.dataflow.sdk.values.TupleTagList; import org.junit.Assert; @@ -55,16 +54,17 @@ public void testRun() throws Exception { PCollectionList list = PCollectionList.of(w1).and(w2); PCollection union = list.apply(Flatten.create()); - PCollectionTuple luc = union.apply(new CountWords(SingletonPCollectionView.of(regex))); - PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); + PCollectionView regexView = regex.apply(View.asSingleton()); + PCollectionTuple luc = union.apply(new CountWords(regexView)); + //PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); EvaluationResult res = SparkPipelineRunner.create().run(p); Iterable> actualLower = res.get(luc.get(lowerCnts)); Iterable> actualUpper = res.get(luc.get(upperCnts)); Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); - Iterable actualUniqCount = res.get(unique); - Assert.assertEquals(9, (long) actualUniqCount.iterator().next()); + // Iterable 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); 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 index 278f42c0e7f0..e549a15f990c 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -55,9 +55,10 @@ public void testRun() throws Exception { actualCountSet.add(s); } - Assert.assertEquals(String.format("Actual counts of words [%s] does not equal expected count[%s].", - actualCountSet, EXPECTED_COUNT_SET), - EXPECTED_COUNT_SET, actualCountSet); + Assert.assertEquals(String.format("Actual counts of words [%s] does not equal expected " + + "count[%s].", + actualCountSet, EXPECTED_COUNT_SET), + EXPECTED_COUNT_SET, actualCountSet); } /** From d6523b741052e2f0e78ba16b7a6eceefe9a594de Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Mon, 5 Jan 2015 18:55:39 -0800 Subject: [PATCH 031/137] Fix side input loading and re-enable approxuniq test --- .../com/cloudera/dataflow/spark/BroadcastHelper.java | 7 +++---- .../com/cloudera/dataflow/spark/EvaluationContext.java | 9 +++++++-- .../dataflow/spark/MultiOutputWordCountTest.java | 9 +++++---- 3 files changed, 15 insertions(+), 10 deletions(-) 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 index edad6692c9c2..75c73a43c396 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -28,11 +28,10 @@ class BroadcastHelper implements Serializable { private static Logger LOG = Logger.getLogger(BroadcastHelper.class.getName()); private Broadcast bcast; private final Coder coder; - private final T input; private transient T value; - BroadcastHelper(T input, Coder coder) { - this.input = input; + BroadcastHelper(T value, Coder coder) { + this.value = value; this.coder = coder; } @@ -44,7 +43,7 @@ public synchronized T getValue() { } public void broadcast(JavaSparkContext jsc) { - this.bcast = jsc.broadcast(CoderHelpers.toByteArray(input, coder)); + this.bcast = jsc.broadcast(CoderHelpers.toByteArray(value, coder)); } private T deserialize() { 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 index 6d51a8b12a15..00a6cff1937b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -71,8 +72,12 @@ SparkRuntimeContext getRuntimeContext() { Coder getDefaultCoder(T example) { Coder defaultCoder = registry.getDefaultCoder(example); if (defaultCoder == null) { - throw new IllegalStateException(String.format("Couldn't determine the default coder for " + - "an example of class [%s]", example.getClass())); + if (example instanceof Iterable) { + return (Coder) IterableCoder.of(getDefaultCoder(((Iterable) example).iterator().next())); + } else { + throw new IllegalStateException(String.format("Couldn't determine the default coder for " + + "an example of class [%s]", example.getClass())); + } } return defaultCoder; } 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 index 74edc1d29749..0c2782f5191f 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; 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; @@ -56,15 +57,15 @@ public void testRun() throws Exception { PCollection union = list.apply(Flatten.create()); PCollectionView regexView = regex.apply(View.asSingleton()); PCollectionTuple luc = union.apply(new CountWords(regexView)); - //PCollection unique = luc.get(lowerCnts).apply(ApproximateUnique.>globally(16)); + PCollection unique = luc.get(lowerCnts).apply( + ApproximateUnique.>globally(16)); EvaluationResult res = SparkPipelineRunner.create().run(p); Iterable> actualLower = res.get(luc.get(lowerCnts)); Iterable> actualUpper = res.get(luc.get(upperCnts)); Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); - // Iterable actualUniqCount = res.get(unique); - //Assert.assertEquals(9, (long) actualUniqCount.iterator().next()); + Iterable 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); From e7982621e7a5c8c8c04c13f0e49d383eb81a28bb Mon Sep 17 00:00:00 2001 From: dhuo Date: Fri, 16 Jan 2015 09:49:19 -0800 Subject: [PATCH 032/137] Add a close() method to EvaluationResult/EvaluationContext which stops the underlying JavaSparkContext (necessary for ensuring Spark EventLogs are flushed before exiting). ------------- Add BUILD rules for MOE GenInternalCodebase. ------------- Fix up MOE inclusions/exclusions. ------------- Add support for handling certain composite PTransforms via single more-suitable TransformEvaluators; map Combine.PerKey to Spark's combineByKey (with some necessary remappings to make the translation work). ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84139560 Signed-off-by: Josh Wills --- .../dataflow/spark/EvaluationContext.java | 5 ++ .../dataflow/spark/EvaluationResult.java | 7 ++ .../dataflow/spark/SparkPipelineRunner.java | 39 ++++++++++ .../dataflow/spark/TransformTranslator.java | 74 +++++++++++++++++++ 4 files changed, 125 insertions(+) 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 index 00a6cff1937b..b34200071808 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -169,4 +169,9 @@ public T apply(byte[] bytes) { } }); } + + @Override + public void close() { + jsc.stop(); + } } 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 index 871a0c100f95..43da7dc5d4d7 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -51,4 +51,11 @@ public interface EvaluationResult extends PipelineResult { * @return Result of aggregation associated with specified name. */ T getAggregatorValue(String aggName, Class resultType); + + /** + * Releases any runtime resources, including distributed-execution contexts currently held by + * this EvaluationResult; once close() has been called, {@link get(PCollection)} might + * not work for subsequent calls. + */ + void close(); } 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 index b0a7c4513ba3..34fcff18030b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -99,20 +99,59 @@ private static class Evaluator implements Pipeline.PipelineVisitor { private final EvaluationContext ctxt; + // Set upon entering a composite node which can be directly mapped to a single + // TransformEvaluator. + private TransformTreeNode currentTranslatedCompositeNode = null; + private Evaluator(EvaluationContext ctxt) { this.ctxt = ctxt; } + /** + * 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()) { + return; + } + + if (node.getTransform() != null + && TransformTranslator.hasTransformEvaluator(node.getTransform().getClass())) { + LOG.info(String.format( + "Entering directly-translatable composite transform: '%s'", node.getFullName())); + LOG.fine(String.format( + "Composite transform class: '%s'", node.getTransform().getClass())); + 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(String.format( + "Post-visiting directly-translatable composite transform: '%s'", node.getFullName())); + doVisitTransform(node.getTransform()); + currentTranslatedCompositeNode = null; + } } @Override public void visitTransform(TransformTreeNode node) { + if (inTranslatedCompositeNode()) { + LOG.info(String.format( + "Skipping '%s'; already in composite transform.", node.getFullName())); + return; + } doVisitTransform(node.getTransform()); } 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 index 199480289479..4bf50d2e7fab 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -33,12 +33,14 @@ 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.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; 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.function.Function; +import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; @@ -121,6 +123,73 @@ public void evaluate(Combine.GroupedValues transform, EvaluationConte }; } + private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class); + + private static TransformEvaluator> combinePerKey() { + return new TransformEvaluator>() { + @Override + public void evaluate(Combine.PerKey transform, EvaluationContext context) { + final Combine.KeyedCombineFn keyed = + COMBINE_PERKEY_FG.get("fn", transform); + @SuppressWarnings("unchecked") + JavaRDDLike, ?> inRdd = + (JavaRDDLike, ?>) context.getInputRDD(transform); + + // 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. + JavaPairRDD> inRddDuplicatedKeyPair = inRdd.mapToPair( + new PairFunction, K, KV>() { + @Override + public Tuple2> call(KV kv) { + return new Tuple2<>(kv.getKey(), kv); + } + }); + + // 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> accumulated = inRddDuplicatedKeyPair.combineByKey( + new Function, KV>() { + @Override + public KV call(KV input) { + VA acc = keyed.createAccumulator(input.getKey()); + keyed.addInput(input.getKey(), acc, input.getValue()); + return KV.of(input.getKey(), acc); + } + }, + new Function2, KV, KV>() { + @Override + public KV call(KV acc, KV input) { + keyed.addInput(acc.getKey(), acc.getValue(), input.getValue()); + return acc; + } + }, + new Function2, KV, KV>() { + @Override + public KV call(KV acc1, KV acc2) { + return KV.of( + acc1.getKey(), + keyed.mergeAccumulators( + acc1.getKey(), ImmutableList.of(acc1.getValue(), acc2.getValue()))); + } + }); + + JavaPairRDD extracted = accumulated.mapValues( + new Function, VO>() { + @Override + public VO call(KV acc) { + return keyed.extractOutput(acc.getKey(), acc.getValue()); + } + }); + context.setOutputRDD(transform, fromPair(extracted)); + } + }; + } + private static final class KVFunction implements Function>, KV> { private final Combine.KeyedCombineFn keyed; @@ -348,6 +417,7 @@ private static Map, BroadcastHelper> getSideInputs( mEvaluators.put(ParDo.BoundMulti.class, multiDo()); mEvaluators.put(GroupByKey.GroupByKeyOnly.class, gbk()); mEvaluators.put(Combine.GroupedValues.class, grouped()); + mEvaluators.put(Combine.PerKey.class, combinePerKey()); mEvaluators.put(Flatten.FlattenPCollectionList.class, flattenPColl()); mEvaluators.put(Create.class, create()); mEvaluators.put(View.AsSingleton.class, viewAsSingleton()); @@ -355,6 +425,10 @@ private static Map, BroadcastHelper> getSideInputs( mEvaluators.put(View.CreatePCollectionView.class, createPCollView()); } + public static boolean hasTransformEvaluator(Class clazz) { + return mEvaluators.containsKey(clazz); + } + public static TransformEvaluator getTransformEvaluator(Class clazz) { @SuppressWarnings("unchecked") From 4cd1a1cffe0d87a37010888d518d2f58ae4e72dc Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Fri, 16 Jan 2015 16:29:43 -0800 Subject: [PATCH 033/137] Add tests for CombinePerKey transform and update other tests to use EvaluationResult.close() method to safely clean up SparkContext --- .../dataflow/spark/EvaluationResult.java | 3 +- .../dataflow/spark/CombinePerKeyTest.java | 49 +++++++++++++++++++ .../spark/MultiOutputWordCountTest.java | 3 +- .../dataflow/spark/SimpleWordCountTest.java | 1 + 4 files changed, 54 insertions(+), 2 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.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 index 43da7dc5d4d7..b40d44978707 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java @@ -54,7 +54,8 @@ public interface EvaluationResult extends PipelineResult { /** * Releases any runtime resources, including distributed-execution contexts currently held by - * this EvaluationResult; once close() has been called, {@link get(PCollection)} might + * this EvaluationResult; once close() has been called, + * {@link com.cloudera.dataflow.spark.EvaluationResult#get(PCollection)} might * not work for subsequent calls. */ void close(); 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 new file mode 100644 index 000000000000..4a70b66a263c --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java @@ -0,0 +1,49 @@ +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.*; +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 List WORDS = ImmutableList.of("the", "quick", "brown", "fox", "jumped", + "over", "the", "lazy", "dog"); + @Test + public void testRun() { + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + PCollection> cnts = inputWords.apply(new SumPerKey()); + EvaluationResult res = SparkPipelineRunner.create().run(p); + Map actualCnts = new HashMap<>(); + for (KV 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 extends PTransform, PCollection>> { + public PCollection> apply(PCollection pcol) { + PCollection> withLongs = pcol.apply(ParDo.of(new DoFn>() { + @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.longsPerKey()); + } + } +} 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 index 0c2782f5191f..57ea55a88a68 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -54,7 +54,7 @@ public void testRun() throws Exception { PCollection w2 = p.apply(Create.of("Here are some more words", "and even more words")); PCollectionList list = PCollectionList.of(w1).and(w2); - PCollection union = list.apply(Flatten.create()); + PCollection union = list.apply(Flatten.pCollections()); PCollectionView regexView = regex.apply(View.asSingleton()); PCollectionTuple luc = union.apply(new CountWords(regexView)); PCollection unique = luc.get(lowerCnts).apply( @@ -70,6 +70,7 @@ public void testRun() throws Exception { Assert.assertEquals(18, actualTotalWords); int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class); Assert.assertEquals(6, actualMaxWordLength); + res.close(); } /** 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 index e549a15f990c..2628a6fa582a 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -59,6 +59,7 @@ public void testRun() throws Exception { "count[%s].", actualCountSet, EXPECTED_COUNT_SET), EXPECTED_COUNT_SET, actualCountSet); + res.close(); } /** From 8a4daa235184a1d6507cf6ca1535a2b5cfdde960 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 26 Jan 2015 22:23:35 +0000 Subject: [PATCH 034/137] Rename artifact to spark-dataflow; add release plugin config; add RAT check and some copyright headers; fix one compiler warning --- runners/spark/README.md | 2 +- runners/spark/build-resources/checkstyle.xml | 12 +++ runners/spark/pom.xml | 101 +++++++++++++++++- .../dataflow/spark/EvaluationContext.java | 5 +- .../dataflow/spark/CombinePerKeyTest.java | 15 +++ .../TestSparkPipelineOptionsFactory.java | 15 +++ 6 files changed, 146 insertions(+), 4 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index f248bc28b7dc..67fec6f273a4 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -12,7 +12,7 @@ program by calling `SparkPipelineRunner.run`. The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark - dataflow-spark + spark-dataflow 0.0.1 If we wanted to run a dataflow pipeline with the default options of a single threaded spark diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml index 4b73e54aaca3..4800f3c57fc5 100644 --- a/runners/spark/build-resources/checkstyle.xml +++ b/runners/spark/build-resources/checkstyle.xml @@ -2,8 +2,20 @@ + - + 4.0.0 Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.0.1-SNAPSHOT + 0.0.1 jar @@ -263,7 +261,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.0.1 From 6ea0b52eba82fdc25494bdde3e831b6dc1d4cdb1 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 26 Jan 2015 22:47:27 +0000 Subject: [PATCH 037/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index bc65d11be349..7800066c3870 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.0.1 + 0.0.2-SNAPSHOT jar @@ -261,7 +261,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.0.1 + HEAD From 3cc19b54f1dad6c7e10bd6e0a54c7e035f757281 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 26 Jan 2015 22:59:03 +0000 Subject: [PATCH 038/137] Add Cloudera repo info --- runners/spark/README.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/runners/spark/README.md b/runners/spark/README.md index 67fec6f273a4..49e450c79ec6 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -14,6 +14,13 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow 0.0.1 + +and are hosted in Cloudera's repository at: + + + cloudera.repo + https://repository.cloudera.com/artifactory/cloudera-repos + 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: From 58228c20422c1038eb7f3f95f290b01164db010a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 26 Jan 2015 22:59:25 +0000 Subject: [PATCH 039/137] Fix formatting of cloudera repo --- runners/spark/README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 49e450c79ec6..708ed1fc894a 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -17,10 +17,10 @@ The Maven coordinates of the current version of this project are: and are hosted in Cloudera's repository at: - - cloudera.repo - https://repository.cloudera.com/artifactory/cloudera-repos - + + cloudera.repo + https://repository.cloudera.com/artifactory/cloudera-repos + 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: From eee09b266954ebe2c75fb72f871176f4fad6d453 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Tue, 27 Jan 2015 09:17:16 -0800 Subject: [PATCH 040/137] Update README w/some notes on motivation. Fixes #20. --- runners/spark/README.md | 32 +++++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 708ed1fc894a..611c062e50b4 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -1,13 +1,34 @@ spark-dataflow ============== -Spark-dataflow is an early prototype. If this project interests you, you should know that we -encourage outside contributions. So, hack away! To get an idea of what we have already identified as +## Intro + +Spark-dataflow allows users to execute data pipelines written against the Google Cloud Dataflow API +with Apache Spark. Spark-dataflow is an early prototype, and we'll be working on it continuously. +If this project interests you, we welcome issues, comments, and (especially!) pull requests. +To get an idea of what we have already identified as areas that need improvement, checkout the issues listed in the github repo. -Spark-dataflow allows users to execute dataflow pipelines with Spark. Executing a pipeline on a -spark cluster is easy: Depend on spark-dataflow in your project and execute your pipeline in a -program by calling `SparkPipelineRunner.run`. +## Motivation + +We had two primary goals when we started working on Spark-dataflow: + +1. *Provide portability for data pipelines written for Google Cloud Dataflow.* Google makes +it really easy to get started writing pipelines against the Dataflow API, but they wanted +to be sure that creating a pipeline using their tools would not lock developers in to their +platform. A Spark-based implementation of Dataflow means that you can take your pipeline +logic with you wherever you go. This also means that any new machine learning and anomaly +detection algorithms that are developed against the Dataflow API are available to everyone, +regardless of their underlying execution platform. + +2. *Experiment with new data pipeline design patterns.* The Dataflow API has a number of +interesting ideas, especially with respect to the unification of batch and stream data +processing into a single API that maps into two separate engines. The Dataflow streaming +engine, based on Google's [Millwheel](http://research.google.com/pubs/pub41378.html), does +not have a direct open source analogue, and we wanted to understand how to replicate its +functionality using frameworks like Spark Streaming. + +## Getting Started The Maven coordinates of the current version of this project are: @@ -35,3 +56,4 @@ would do the following: SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); options.setSparkMaster("spark://host:port"); EvaluationResult result = SparkPipelineRunner.create(options).run(p); + From 68b72e793c829dccad305b343c49a42de6edc952 Mon Sep 17 00:00:00 2001 From: David Alves Date: Wed, 4 Feb 2015 23:36:06 -0800 Subject: [PATCH 041/137] Added new test for TransformTranslator and tested TextIO.Read and TextIO.Write transforms This adds a TranformTranslatorTest that creates a pipeline and then runs it in the direct runner and in the spark runner, making sure that the results are the same. Specifically, this commit adds a test for the TextIO transforms. --- .../spark/TransformTranslatorTest.java | 93 +++++++++++++++++++ .../spark/src/test/resources/test_text.txt | 2 + 2 files changed, 95 insertions(+) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java create mode 100644 runners/spark/src/test/resources/test_text.txt 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 new file mode 100644 index 000000000000..3547d1a43565 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.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 com.cloudera.dataflow.spark; + +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.DirectPipelineRunner.EvaluationResults; +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.List; + +import static com.google.api.client.repackaged.com.google.common.base.Joiner.on; +import static java.io.File.separator; + +/** + * 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 Pipeline testPipeline; + private DirectPipelineRunner directRunner; + private SparkPipelineRunner sparkRunner; + private EvaluationResults directRunResult; + private EvaluationResult sparkRunResult; + private String testDataDirName; + + @Before public void init() throws IOException { + testPipeline = Pipeline.create(PipelineOptionsFactory.create()); + sparkRunner = SparkPipelineRunner.create(); + directRunner = DirectPipelineRunner.createForTest(); + testDataDirName = on(separator).join("target", "test-data", name.getMethodName()) + separator; + FileUtils.deleteDirectory(new File(testDataDirName)); + new File(testDataDirName).mkdirs(); + } + + public void run() { + directRunResult = directRunner.run(testPipeline); + sparkRunResult = sparkRunner.run(testPipeline); + } + + /** + * 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 outFile = on(separator).join(testDataDirName, "test_text_out"); + PCollection lines = testPipeline + .apply(TextIO.Read.from("src/test/resources/test_text.txt")); + lines.apply(TextIO.Write.to(outFile)); + run(); + + List directOutput = Files.readAllLines(Paths.get(outFile + "-00000-of-00001"), + Charsets.UTF_8); + + List sparkOutput = Files.readAllLines(Paths.get( + on(separator).join(outFile, "part-00000")), + Charsets.UTF_8); + + Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray()); + } +} diff --git a/runners/spark/src/test/resources/test_text.txt b/runners/spark/src/test/resources/test_text.txt new file mode 100644 index 000000000000..6a14a1b0fe8e --- /dev/null +++ b/runners/spark/src/test/resources/test_text.txt @@ -0,0 +1,2 @@ +test line 1 +test line 2 \ No newline at end of file From b77179adfd53b6a70d452ee08a165ce5b5fc7c45 Mon Sep 17 00:00:00 2001 From: Josh Wills Date: Fri, 10 Apr 2015 08:46:27 -0700 Subject: [PATCH 042/137] Update to latest version of Dataflow APIs and mimic createAggregator implementation model in Spark Dataflow. --- runners/spark/pom.xml | 2 +- .../cloudera/dataflow/spark/DoFnFunction.java | 13 ++++- .../dataflow/spark/EvaluationContext.java | 2 +- .../dataflow/spark/MultiDoFnFunction.java | 7 ++- .../dataflow/spark/SparkRuntimeContext.java | 47 +++++++++++++++++-- .../dataflow/spark/TransformTranslator.java | 11 ++--- .../spark/aggregators/NamedAggregators.java | 33 +++++++++++-- .../spark/MultiOutputWordCountTest.java | 10 ++-- 8 files changed, 103 insertions(+), 22 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 7800066c3870..065a5405c65b 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -203,7 +203,7 @@ License. com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - 0.3.141216 + 0.3.150326 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 index 85a9cab92c94..489f0a7fb3ee 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.AggregatorImpl; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; @@ -91,7 +92,7 @@ public PipelineOptions getPipelineOptions() { } @Override - public T sideInput(PCollectionView view) { + public T sideInput(PCollectionView view) { @SuppressWarnings("unchecked") T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); return value; @@ -104,12 +105,20 @@ public synchronized void output(O o) { @Override public void sideOutput(TupleTag tupleTag, T t) { - String message = "sideOutput is an unsupported operation for doFunctions, use a" + + String message = "sideOutput is an unsupported operation for doFunctions, use a " + "MultiDoFunction instead."; LOG.warning(message); throw new UnsupportedOperationException(message); } + @Override + public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { + String message = "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " + + "MultiDoFunction instead."; + LOG.warning(message); + throw new UnsupportedOperationException(message); + } + @Override public Aggregator createAggregator( String named, 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 index 572c2063056a..bbcd053545bd 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -131,7 +131,7 @@ void setRDD(PValue pvalue, JavaRDDLike rdd) { } - Iterable> getPCollectionView(PCollectionView view) { + Iterable> getPCollectionView(PCollectionView view) { Iterable> value = pview.get(view); return value; } 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 index 60447f89f83f..adcb4a264ec2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -97,7 +97,7 @@ public PipelineOptions getPipelineOptions() { } @Override - public T sideInput(PCollectionView view) { + public T sideInput(PCollectionView view) { @SuppressWarnings("unchecked") T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); return value; @@ -113,6 +113,11 @@ public synchronized void sideOutput(TupleTag tag, T t) { outputs.put(tag, t); } + @Override + public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { + outputs.put(tupleTag, t); + } + @Override public Aggregator createAggregator( String named, 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 index 198f0d65aedb..8c721a83a5c4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -20,10 +20,17 @@ import java.util.Map; import com.google.cloud.dataflow.sdk.Pipeline; +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.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.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.common.reflect.TypeToken; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; @@ -34,7 +41,7 @@ * The SparkRuntimeContext allows us to define useful features on the client side before our * data flow program is launched. */ -class SparkRuntimeContext implements Serializable { +public class SparkRuntimeContext implements Serializable { /** * An accumulator that is a map from names to aggregators. */ @@ -43,6 +50,7 @@ class SparkRuntimeContext implements Serializable { * Map fo names to dataflow aggregators. */ private final Map> aggregators = new HashMap<>(); + private transient CoderRegistry coderRegistry; SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); @@ -106,7 +114,7 @@ public synchronized Aggregator createAggregator( if (aggregator == null) { @SuppressWarnings("unchecked") NamedAggregators.CombineFunctionState state = new NamedAggregators - .CombineFunctionState<>((Combine.CombineFn) combineFn); + .CombineFunctionState<>((Combine.CombineFn) combineFn, (Coder) getCoder(combineFn), this); accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator<>(state); aggregators.put(named, aggregator); @@ -114,12 +122,45 @@ public synchronized Aggregator createAggregator( return aggregator; } + public CoderRegistry getCoderRegistry() { + if (coderRegistry == null) { + coderRegistry = new CoderRegistry(); + coderRegistry.registerStandardCoders(); + } + return coderRegistry; + } + + private Coder getCoder(Combine.CombineFn combiner) { + if (combiner.getClass() == Sum.SumIntegerFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); + } else if (combiner.getClass() == Sum.SumLongFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); + } else if (combiner.getClass() == Sum.SumDoubleFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); + } else if (combiner.getClass() == Min.MinIntegerFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); + } else if (combiner.getClass() == Min.MinLongFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); + } else if (combiner.getClass() == Min.MinDoubleFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); + } else if (combiner.getClass() == Max.MaxIntegerFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); + } else if (combiner.getClass() == Max.MaxLongFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); + } else if (combiner.getClass() == Max.MaxDoubleFn.class) { + return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); + } else { + throw new IllegalArgumentException("unsupported combiner in Aggregator: " + + combiner.getClass().getName()); + } + } + /** * Initialize spark aggregators exactly once. * * @param Type of element fed in to aggregator. */ - private static class SparkAggregator implements Aggregator { + private static class SparkAggregator implements Aggregator, Serializable { private final NamedAggregators.State state; SparkAggregator(NamedAggregators.State state) { 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 index 4bf50d2e7fab..16da55831169 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -350,11 +350,10 @@ public void evaluate(View.AsIterable transform, EvaluationContext context) { }; } - private static TransformEvaluator> createPCollView() { - return new TransformEvaluator>() { + private static TransformEvaluator> createPCollView() { + return new TransformEvaluator>() { @Override - public void evaluate(View.CreatePCollectionView transform, EvaluationContext + public void evaluate(View.CreatePCollectionView transform, EvaluationContext context) { Iterable> iter = Iterables.transform(context.get(context.getInput (transform)), new WindowingFunction() @@ -387,13 +386,13 @@ public Boolean call(Tuple2, Object> input) { } private static Map, BroadcastHelper> getSideInputs( - List> views, + List> views, EvaluationContext context) { if (views == null) { return ImmutableMap.of(); } else { Map, BroadcastHelper> sideInputs = Maps.newHashMap(); - for (PCollectionView view : views) { + for (PCollectionView view : views) { Object sideinput = view.fromIterableInternal(context.getPCollectionView(view)); Coder coder = context.getDefaultCoder(sideinput); BroadcastHelper helper = new BroadcastHelper<>(sideinput, coder); 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 index fde44ff4c8d3..28db4d9d0f2f 100644 --- 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 @@ -15,10 +15,16 @@ package com.cloudera.dataflow.spark.aggregators; +import com.cloudera.dataflow.spark.SparkRuntimeContext; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; @@ -124,11 +130,18 @@ public interface State extends Serializable { */ public static class CombineFunctionState implements State { - private final Combine.CombineFn combineFn; - private Inter state; + private Combine.CombineFn combineFn; + private Coder inCoder; + private SparkRuntimeContext ctxt; + private transient Inter state; - public CombineFunctionState(Combine.CombineFn combineFn) { + public CombineFunctionState( + Combine.CombineFn combineFn, + Coder inCoder, + SparkRuntimeContext ctxt) { this.combineFn = combineFn; + this.inCoder = inCoder; + this.ctxt = ctxt; this.state = combineFn.createAccumulator(); } @@ -152,6 +165,20 @@ public Inter current() { public Out render() { return combineFn.extractOutput(state); } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.writeObject(ctxt); + oos.writeObject(combineFn); + oos.writeObject(inCoder); + combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder).encode(state, oos, Coder.Context.NESTED); + } + + private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { + ctxt = (SparkRuntimeContext) ois.readObject(); + combineFn = (Combine.CombineFn) ois.readObject(); + inCoder = (Coder) ois.readObject(); + state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder).decode(ois, Coder.Context.NESTED); + } } /** 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 index 57ea55a88a68..67c9b883470b 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -55,7 +55,7 @@ public void testRun() throws Exception { PCollectionList list = PCollectionList.of(w1).and(w2); PCollection union = list.apply(Flatten.pCollections()); - PCollectionView regexView = regex.apply(View.asSingleton()); + PCollectionView regexView = regex.apply(View.asSingleton()); PCollectionTuple luc = union.apply(new CountWords(regexView)); PCollection unique = luc.get(lowerCnts).apply( ApproximateUnique.>globally(16)); @@ -80,9 +80,9 @@ static class ExtractWordsFn extends DoFn { private Aggregator totalWords; private Aggregator maxWordLength; - private final PCollectionView regex; + private final PCollectionView regex; - ExtractWordsFn(PCollectionView regex) { + ExtractWordsFn(PCollectionView regex) { this.regex = regex; } @@ -113,9 +113,9 @@ public void processElement(ProcessContext c) { public static class CountWords extends PTransform, PCollectionTuple> { - private final PCollectionView regex; + private final PCollectionView regex; - public CountWords(PCollectionView regex) { + public CountWords(PCollectionView regex) { this.regex = regex; } From d5aefefa428c157e0ee78abf6f9548a5577f8867 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 28 Apr 2015 15:22:09 +0100 Subject: [PATCH 043/137] Add a test for Avro. --- runners/spark/pom.xml | 6 + .../dataflow/spark/TransformTranslator.java | 37 ++++++- .../dataflow/spark/AvroPipelineTest.java | 104 ++++++++++++++++++ runners/spark/src/test/resources/person.avsc | 23 ++++ 4 files changed, 167 insertions(+), 3 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java create mode 100644 runners/spark/src/test/resources/person.avsc diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 065a5405c65b..33ec34ed4680 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -205,6 +205,12 @@ License. google-cloud-dataflow-java-sdk-all 0.3.150326 + + org.apache.avro + avro-mapred + 1.7.7 + hadoop2 + 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 index 16da55831169..5f4a721e1b02 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -36,11 +36,20 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import java.io.IOException; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroJob; +import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.avro.mapreduce.AvroKeyOutputFormat; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Job; 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.function.Function; import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; @@ -295,7 +304,15 @@ private static TransformEvaluator> readAvro() { @Override public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); + JavaRDD rdd = context.getSparkContext() + .newAPIHadoopFile(pattern, AvroKeyInputFormat.class, AvroKey.class, + NullWritable.class, new Configuration()) + .map(new Function, Object>() { + @Override + public Object call(Tuple2 t) throws Exception { + return t._1().datum(); + } + }); context.setOutputRDD(transform, rdd); } }; @@ -306,9 +323,23 @@ private static TransformEvaluator> writeAvro() { @Override public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { @SuppressWarnings("unchecked") - JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); String pattern = transform.getFilenamePrefix(); - last.saveAsTextFile(pattern); + JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); + Job job; + try { + job = Job.getInstance(); + } catch (IOException e) { + throw new RuntimeException(e); + } + AvroJob.setOutputKeySchema(job, transform.getSchema()); + last.mapToPair(new PairFunction, NullWritable>() { + @Override + public Tuple2, NullWritable> call(T t) throws Exception { + return new Tuple2<>(new AvroKey<>(t), NullWritable.get()); + }}) + .saveAsNewAPIHadoopFile(pattern, AvroKey.class, NullWritable.class, + AvroKeyOutputFormat.class, job.getConfiguration()); + } }; } 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 new file mode 100644 index 000000000000..8cb8bb3f116e --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java @@ -0,0 +1,104 @@ +/* + * 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 transient File inputFile; + private transient File outputDir; + + @Rule + public transient 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 input = p.apply( + AvroIO.Read.from(inputFile.getAbsolutePath()).withSchema(schema)); + input.apply(AvroIO.Write.to(outputDir.getAbsolutePath()).withSchema(schema)); + SparkPipelineRunner.create().run(p); + + List records = readGenericFile(); + assertEquals(Lists.newArrayList(savedRecord), records); + } + + private void populateGenericFile(List genericRecords, Schema schema) throws IOException { + FileOutputStream outputStream = new FileOutputStream(this.inputFile); + GenericDatumWriter genericDatumWriter = new GenericDatumWriter(schema); + + DataFileWriter dataFileWriter = new DataFileWriter(genericDatumWriter); + dataFileWriter.create(schema, outputStream); + + for (GenericRecord record : genericRecords) { + dataFileWriter.append(record); + } + + dataFileWriter.close(); + outputStream.close(); + } + + private List readGenericFile() throws IOException { + List records = Lists.newArrayList(); + GenericDatumReader genericDatumReader = new GenericDatumReader(); + DataFileReader dataFileReader = new DataFileReader + (new File(outputDir, "part-r-00000.avro"), genericDatumReader); + for (GenericRecord record : dataFileReader) { + records.add(record); + } + dataFileReader.close(); + return records; + } + + +} diff --git a/runners/spark/src/test/resources/person.avsc b/runners/spark/src/test/resources/person.avsc new file mode 100644 index 000000000000..c20797d2dde1 --- /dev/null +++ b/runners/spark/src/test/resources/person.avsc @@ -0,0 +1,23 @@ +/* + * 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. + */ +{ +"namespace": "com.cloudera.dataflow.spark.test", +"name": "Person", +"type": "record", +"fields": [ + {"name": "name", "type": ["string", "null"] }, + {"name": "age", "type": "int"}, + {"name": "siblingnames", "type" : ["null", { "type": "array", "items": "string" }], "default": null } ] +} From 60a4c25ea28e21cd85eb242830920b1491c9240e Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 11 May 2015 14:05:16 +0100 Subject: [PATCH 044/137] Add test that uses DeDupExample from the SDK. --- .../cloudera/dataflow/spark/DeDupTest.java | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 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 new file mode 100644 index 000000000000..e26a50854995 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java @@ -0,0 +1,44 @@ +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.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.HashSet; +import java.util.List; +import java.util.Set; +import org.junit.Assert; +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 LINES = Arrays.asList(LINES_ARRAY); + private static final Set EXPECTED_SET = + ImmutableSet.of("hi there", "hi", "hello"); + + @Test + public void testRun() throws Exception { + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection input = p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of()); + PCollection output = input.apply(RemoveDuplicates.create()); + EvaluationResult res = SparkPipelineRunner.create().run(p); + Set actualSet = new HashSet<>(); + for (String s : res.get(output)) { + actualSet.add(s); + } + + Assert.assertEquals(String.format("Actual lines [%s] does not equal expected [%s].", + actualSet, EXPECTED_SET), EXPECTED_SET, actualSet); + res.close(); + } +} From ba882f2ad0b9d7379aa4ad5d81f05a9826dd5404 Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 11 May 2015 15:03:07 +0100 Subject: [PATCH 045/137] Add test that uses TfIdf from the SDK and switch to use KryoSerializer. --- .../dataflow/spark/SparkPipelineRunner.java | 2 + .../cloudera/dataflow/spark/TfIdfTest.java | 401 ++ runners/spark/src/test/resources/pg1112.txt | 4853 +++++++++++++++++ runners/spark/src/test/resources/pg2264.txt | 3667 +++++++++++++ 4 files changed, 8923 insertions(+) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java create mode 100644 runners/spark/src/test/resources/pg1112.txt create mode 100644 runners/spark/src/test/resources/pg2264.txt 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 index 68372189b31c..654b7e289927 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -24,6 +24,7 @@ import org.apache.spark.api.java.JavaSparkContext; import java.util.logging.Logger; +import org.apache.spark.serializer.KryoSerializer; /** * The SparkPipelineRunner translate operations defined on a pipeline to a representation @@ -98,6 +99,7 @@ private JavaSparkContext getContext() { conf.setMaster(mOptions.getSparkMaster()); conf.setAppName("spark pipeline job"); conf.set("sun.io.serialization.extendeddebuginfo", "true"); + conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); return new JavaSparkContext(conf); } 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 new file mode 100644 index 000000000000..38dfd615fe12 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -0,0 +1,401 @@ +package com.cloudera.dataflow.spark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.Keys; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.transforms.Values; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.WithKeys; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +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.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import com.google.common.io.Resources; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * A test based on {@code TfIdf} from the SDK. + */ +public class TfIdfTest { + + private transient File inputDir; + private transient File outputDir; + + @Rule + public transient TemporaryFolder tmpDir = new TemporaryFolder(); + + @Before + public void setUp() throws IOException { + inputDir = tmpDir.newFolder("in"); + copy("pg1112.txt", inputDir); + copy("pg2264.txt", inputDir); + outputDir = tmpDir.newFolder("out"); + outputDir.delete(); + } + + private void copy(String resourceName, File dir) throws IOException { + File dest = new File(dir, resourceName); + Resources.copy(Resources.getResource(resourceName), Files.asByteSink(dest).openStream()); + } + + @Test + public void testRun() throws Exception { + Pipeline pipeline = Pipeline.create(PipelineOptionsFactory.create()); + + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + + pipeline + .apply(new ReadDocuments(listInputDocuments())) + .apply(new ComputeTfIdf()) + .apply(new WriteTfIdf(outputDir.toURI().toString())); + + SparkPipelineRunner.create().run(pipeline); + + for (File f : outputDir.listFiles(new FileFilter() { + @Override public boolean accept(File pathname) { + return pathname.getName().startsWith("part-"); + } + })) { + for (String line : Files.readLines(f, Charsets.UTF_8)) { + if (line.startsWith("love,")) { + assertTrue("Incorrect term frequency", line.endsWith("0.003484")); + return; + } + } + + } + fail("Could not find 'love' in output."); + } + + public Set listInputDocuments() + throws URISyntaxException, IOException { + Set uris = new HashSet<>(); + File directory = inputDir; + for (String entry : directory.list()) { + File path = new File(directory, entry); + uris.add(path.toURI()); + } + return uris; + } + + /** + * Reads the documents at the provided uris and returns all lines + * from the documents tagged with which document they are from. + */ + public static class ReadDocuments + extends PTransform>> { + private static final long serialVersionUID = 0; + + private Iterable uris; + + public ReadDocuments(Iterable uris) { + this.uris = uris; + } + + @Override + public Coder getDefaultOutputCoder() { + return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); + } + + @Override + public PCollection> apply(PInput input) { + Pipeline pipeline = input.getPipeline(); + + // Create one TextIO.Read transform for each document + // and add its output to a PCollectionList + PCollectionList> urisToLines = + PCollectionList.empty(pipeline); + + // TextIO.Read supports: + // - file: URIs and paths locally + // - gs: URIs on the service + for (final URI uri : uris) { + String uriString; + if (uri.getScheme().equals("file")) { + uriString = new File(uri).getPath(); + } else { + uriString = uri.toString(); + } + + PCollection> oneUriToLines = pipeline + .apply(TextIO.Read.from(uriString) + .named("TextIO.Read(" + uriString + ")")) + .apply(WithKeys.of(uri).withName("WithKeys(" + uriString + ")")); + + urisToLines = urisToLines.and(oneUriToLines); + } + + return urisToLines.apply(Flatten.>create()); + } + } + + /** + * A transform containing a basic TF-IDF pipeline. The input consists of KV objects + * where the key is the document's URI and the value is a piece + * of the document's content. The output is mapping from terms to + * scores for each document URI. + */ + public static class ComputeTfIdf + extends PTransform>, PCollection>>> { + private static final long serialVersionUID = 0; + + public ComputeTfIdf() { } + + @Override + public PCollection>> apply( + PCollection> uriToContent) { + + // Compute the total number of documents, and + // prepare this singleton PCollectionView for + // use as a side input. + final PCollectionView totalDocuments = + uriToContent + .apply(Keys.create().withName("GetURIs")) + .apply(RemoveDuplicates.create().withName("RemoveDuplicateDocs")) + .apply(Count.globally()) + .apply(View.asSingleton()); + + // Create a collection of pairs mapping a URI to each + // of the words in the document associated with that that URI. + PCollection> uriToWords = uriToContent + .apply(ParDo.named("SplitWords").of( + new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + String line = c.element().getValue(); + for (String word : line.split("\\W+")) { + // Log INFO messages when the word “love” is found. + if (word.toLowerCase().equals("love")) { + LOG.info("Found {}", word.toLowerCase()); + } + + if (!word.isEmpty()) { + c.output(KV.of(uri, word.toLowerCase())); + } + } + } + })); + + // Compute a mapping from each word to the total + // number of documents in which it appears. + PCollection> wordToDocCount = uriToWords + .apply(RemoveDuplicates.>create().withName("RemoveDuplicateWords")) + .apply(Values.create()) + .apply(Count.perElement().withName("CountDocs")); + + // Compute a mapping from each URI to the total + // number of words in the document associated with that URI. + PCollection> uriToWordTotal = uriToWords + .apply(Keys.create().withName("GetURIs2")) + .apply(Count.perElement().withName("CountWords")); + + // Count, for each (URI, word) pair, the number of + // occurrences of that word in the document associated + // with the URI. + PCollection, Long>> uriAndWordToCount = uriToWords + .apply(Count.>perElement().withName("CountWordDocPairs")); + + // Adjust the above collection to a mapping from + // (URI, word) pairs to counts into an isomorphic mapping + // from URI to (word, count) pairs, to prepare for a join + // by the URI key. + PCollection>> uriToWordAndCount = uriAndWordToCount + .apply(ParDo.named("ShiftKeys").of( + new DoFn, Long>, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey().getKey(); + String word = c.element().getKey().getValue(); + Long occurrences = c.element().getValue(); + c.output(KV.of(uri, KV.of(word, occurrences))); + } + })); + + // Prepare to join the mapping of URI to (word, count) pairs with + // the mapping of URI to total word counts, by associating + // each of the input PCollection> with + // a tuple tag. Each input must have the same key type, URI + // in this case. The type parameter of the tuple tag matches + // the types of the values for each collection. + final TupleTag wordTotalsTag = new TupleTag(); + final TupleTag> wordCountsTag = new TupleTag>(); + KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple + .of(wordTotalsTag, uriToWordTotal) + .and(wordCountsTag, uriToWordAndCount); + + // Perform a CoGroupByKey (a sort of pre-join) on the prepared + // inputs. This yields a mapping from URI to a CoGbkResult + // (CoGroupByKey Result). The CoGbkResult is a mapping + // from the above tuple tags to the values in each input + // associated with a particular URI. In this case, each + // KV group a URI with the total number of + // words in that document as well as all the (word, count) + // pairs for particular words. + PCollection> uriToWordAndCountAndTotal = coGbkInput + .apply(CoGroupByKey.create().withName("CoGroupByURI")); + + // Compute a mapping from each word to a (URI, term frequency) + // pair for each URI. A word's term frequency for a document + // is simply the number of times that word occurs in the document + // divided by the total number of words in the document. + PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal + .apply(ParDo.named("ComputeTermFrequencies").of( + new DoFn, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); + + for (KV wordAndCount + : c.element().getValue().getAll(wordCountsTag)) { + String word = wordAndCount.getKey(); + Long wordCount = wordAndCount.getValue(); + Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); + c.output(KV.of(word, KV.of(uri, termFrequency))); + } + } + })); + + // Compute a mapping from each word to its document frequency. + // A word's document frequency in a corpus is the number of + // documents in which the word appears divided by the total + // number of documents in the corpus. Note how the total number of + // documents is passed as a side input; the same value is + // presented to each invocation of the DoFn. + PCollection> wordToDf = wordToDocCount + .apply(ParDo + .named("ComputeDocFrequencies") + .withSideInputs(totalDocuments) + .of(new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Long documentCount = c.element().getValue(); + Long documentTotal = c.sideInput(totalDocuments); + Double documentFrequency = documentCount.doubleValue() + / documentTotal.doubleValue(); + + c.output(KV.of(word, documentFrequency)); + } + })); + + // Join the term frequency and document frequency + // collections, each keyed on the word. + final TupleTag> tfTag = new TupleTag>(); + final TupleTag dfTag = new TupleTag(); + PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple + .of(tfTag, wordToUriAndTf) + .and(dfTag, wordToDf) + .apply(CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, TF-IDF) score + // for each URI. There are a variety of definitions of TF-IDF + // ("term frequency - inverse document frequency") score; + // here we use a basic version that is the term frequency + // divided by the log of the document frequency. + PCollection>> wordToUriAndTfIdf = wordToUriAndTfAndDf + .apply(ParDo.named("ComputeTfIdf").of( + new DoFn, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Double df = c.element().getValue().getOnly(dfTag); + + for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { + URI uri = uriAndTf.getKey(); + Double tf = uriAndTf.getValue(); + Double tfIdf = tf * Math.log(1 / df); + c.output(KV.of(word, KV.of(uri, tfIdf))); + } + } + })); + + return wordToUriAndTfIdf; + } + + // Instantiate Logger. + // It is suggested that the user specify the class name of the containing class + // (in this case ComputeTfIdf). + private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); + } + + /** + * A {@link PTransform} to write, in CSV format, a mapping from term and URI + * to score. + */ + public static class WriteTfIdf + extends PTransform>>, PDone> { + private static final long serialVersionUID = 0; + + private String output; + + public WriteTfIdf(String output) { + this.output = output; + } + + @Override + public PDone apply(PCollection>> wordToUriAndTfIdf) { + return wordToUriAndTfIdf + .apply(ParDo.named("Format").of(new DoFn>, String> + () { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + c.output(String.format("%s,\t%s,\t%f", + c.element().getKey(), + c.element().getValue().getKey(), + c.element().getValue().getValue())); + } + })) + .apply(TextIO.Write + .to(output) + .withSuffix(".csv")); + } + } +} diff --git a/runners/spark/src/test/resources/pg1112.txt b/runners/spark/src/test/resources/pg1112.txt new file mode 100644 index 000000000000..81ee6b87ea0e --- /dev/null +++ b/runners/spark/src/test/resources/pg1112.txt @@ -0,0 +1,4853 @@ +The Project Gutenberg EBook of Romeo and Juliet, by William Shakespeare + +This eBook is for the use of anyone anywhere at no cost and with +almost no restrictions whatsoever. You may copy it, give it away or +re-use it under the terms of the Project Gutenberg License included +with this eBook or online at www.gutenberg.org/license + + +Title: Romeo and Juliet + +Author: William Shakespeare + +Posting Date: May 25, 2012 [EBook #1112] +Release Date: November, 1997 [Etext #1112] + +Language: English + + +*** START OF THIS PROJECT GUTENBERG EBOOK ROMEO AND JULIET *** + + + + + + + + + + + + + +*Project Gutenberg is proud to cooperate with The World Library* +in the presentation of The Complete Works of William Shakespeare +for your reading for education and entertainment. HOWEVER, THIS +IS NEITHER SHAREWARE NOR PUBLIC DOMAIN. . .AND UNDER THE LIBRARY +OF THE FUTURE CONDITIONS OF THIS PRESENTATION. . .NO CHARGES MAY +BE MADE FOR *ANY* ACCESS TO THIS MATERIAL. YOU ARE ENCOURAGED!! +TO GIVE IT AWAY TO ANYONE YOU LIKE, BUT NO CHARGES ARE ALLOWED!! + + + + +The Complete Works of William Shakespeare + +The Tragedy of Romeo and Juliet + +The Library of the Future Complete Works of William Shakespeare +Library of the Future is a TradeMark (TM) of World Library Inc. + + +<> + + + + +1595 + +THE TRAGEDY OF ROMEO AND JULIET + +by William Shakespeare + + + +Dramatis Personae + + Chorus. + + + Escalus, Prince of Verona. + + Paris, a young Count, kinsman to the Prince. + + Montague, heads of two houses at variance with each other. + + Capulet, heads of two houses at variance with each other. + + An old Man, of the Capulet family. + + Romeo, son to Montague. + + Tybalt, nephew to Lady Capulet. + + Mercutio, kinsman to the Prince and friend to Romeo. + + Benvolio, nephew to Montague, and friend to Romeo + + Tybalt, nephew to Lady Capulet. + + Friar Laurence, Franciscan. + + Friar John, Franciscan. + + Balthasar, servant to Romeo. + + Abram, servant to Montague. + + Sampson, servant to Capulet. + + Gregory, servant to Capulet. + + Peter, servant to Juliet's nurse. + + An Apothecary. + + Three Musicians. + + An Officer. + + + Lady Montague, wife to Montague. + + Lady Capulet, wife to Capulet. + + Juliet, daughter to Capulet. + + Nurse to Juliet. + + + Citizens of Verona; Gentlemen and Gentlewomen of both houses; + Maskers, Torchbearers, Pages, Guards, Watchmen, Servants, and + Attendants. + + SCENE.--Verona; Mantua. + + + + THE PROLOGUE + + Enter Chorus. + + + Chor. Two households, both alike in dignity, + In fair Verona, where we lay our scene, + From ancient grudge break to new mutiny, + Where civil blood makes civil hands unclean. + From forth the fatal loins of these two foes + A pair of star-cross'd lovers take their life; + Whose misadventur'd piteous overthrows + Doth with their death bury their parents' strife. + The fearful passage of their death-mark'd love, + And the continuance of their parents' rage, + Which, but their children's end, naught could remove, + Is now the two hours' traffic of our stage; + The which if you with patient ears attend, + What here shall miss, our toil shall strive to mend. + [Exit.] + + + + +ACT I. Scene I. +Verona. A public place. + +Enter Sampson and Gregory (with swords and bucklers) of the house +of Capulet. + + + Samp. Gregory, on my word, we'll not carry coals. + + Greg. No, for then we should be colliers. + + Samp. I mean, an we be in choler, we'll draw. + + Greg. Ay, while you live, draw your neck out of collar. + + Samp. I strike quickly, being moved. + + Greg. But thou art not quickly moved to strike. + + Samp. A dog of the house of Montague moves me. + + Greg. To move is to stir, and to be valiant is to stand. + Therefore, if thou art moved, thou runn'st away. + + Samp. A dog of that house shall move me to stand. I will take + the wall of any man or maid of Montague's. + + Greg. That shows thee a weak slave; for the weakest goes to the + wall. + + Samp. 'Tis true; and therefore women, being the weaker vessels, + are ever thrust to the wall. Therefore I will push Montague's men + from the wall and thrust his maids to the wall. + + Greg. The quarrel is between our masters and us their men. + + Samp. 'Tis all one. I will show myself a tyrant. When I have + fought with the men, I will be cruel with the maids- I will cut off + their heads. + + Greg. The heads of the maids? + + Samp. Ay, the heads of the maids, or their maidenheads. + Take it in what sense thou wilt. + + Greg. They must take it in sense that feel it. + + Samp. Me they shall feel while I am able to stand; and 'tis known I + am a pretty piece of flesh. + + Greg. 'Tis well thou art not fish; if thou hadst, thou hadst + been poor-John. Draw thy tool! Here comes two of the house of + Montagues. + + Enter two other Servingmen [Abram and Balthasar]. + + + Samp. My naked weapon is out. Quarrel! I will back thee. + + Greg. How? turn thy back and run? + + Samp. Fear me not. + + Greg. No, marry. I fear thee! + + Samp. Let us take the law of our sides; let them begin. + + Greg. I will frown as I pass by, and let them take it as they list. + + Samp. Nay, as they dare. I will bite my thumb at them; which is + disgrace to them, if they bear it. + + Abr. Do you bite your thumb at us, sir? + + Samp. I do bite my thumb, sir. + + Abr. Do you bite your thumb at us, sir? + + Samp. [aside to Gregory] Is the law of our side if I say ay? + + Greg. [aside to Sampson] No. + + Samp. No, sir, I do not bite my thumb at you, sir; but I bite my + thumb, sir. + + Greg. Do you quarrel, sir? + + Abr. Quarrel, sir? No, sir. + + Samp. But if you do, sir, am for you. I serve as good a man as + you. + + Abr. No better. + + Samp. Well, sir. + + Enter Benvolio. + + + Greg. [aside to Sampson] Say 'better.' Here comes one of my + master's kinsmen. + + Samp. Yes, better, sir. + + Abr. You lie. + + Samp. Draw, if you be men. Gregory, remember thy swashing blow. + They fight. + + Ben. Part, fools! [Beats down their swords.] + Put up your swords. You know not what you do. + + Enter Tybalt. + + + Tyb. What, art thou drawn among these heartless hinds? + Turn thee Benvolio! look upon thy death. + + Ben. I do but keep the peace. Put up thy sword, + Or manage it to part these men with me. + + Tyb. What, drawn, and talk of peace? I hate the word + As I hate hell, all Montagues, and thee. + Have at thee, coward! They fight. + + Enter an officer, and three or four Citizens with clubs or + partisans. + + + Officer. Clubs, bills, and partisans! Strike! beat them down! + + Citizens. Down with the Capulets! Down with the Montagues! + + Enter Old Capulet in his gown, and his Wife. + + + Cap. What noise is this? Give me my long sword, ho! + + Wife. A crutch, a crutch! Why call you for a sword? + + Cap. My sword, I say! Old Montague is come + And flourishes his blade in spite of me. + + Enter Old Montague and his Wife. + + + Mon. Thou villain Capulet!- Hold me not, let me go. + + M. Wife. Thou shalt not stir one foot to seek a foe. + + Enter Prince Escalus, with his Train. + + + Prince. Rebellious subjects, enemies to peace, + Profaners of this neighbour-stained steel- + Will they not hear? What, ho! you men, you beasts, + That quench the fire of your pernicious rage + With purple fountains issuing from your veins! + On pain of torture, from those bloody hands + Throw your mistempered weapons to the ground + And hear the sentence of your moved prince. + Three civil brawls, bred of an airy word + By thee, old Capulet, and Montague, + Have thrice disturb'd the quiet of our streets + And made Verona's ancient citizens + Cast by their grave beseeming ornaments + To wield old partisans, in hands as old, + Cank'red with peace, to part your cank'red hate. + If ever you disturb our streets again, + Your lives shall pay the forfeit of the peace. + For this time all the rest depart away. + You, Capulet, shall go along with me; + And, Montague, come you this afternoon, + To know our farther pleasure in this case, + To old Freetown, our common judgment place. + Once more, on pain of death, all men depart. + Exeunt [all but Montague, his Wife, and Benvolio]. + + Mon. Who set this ancient quarrel new abroach? + Speak, nephew, were you by when it began? + + Ben. Here were the servants of your adversary + And yours, close fighting ere I did approach. + I drew to part them. In the instant came + The fiery Tybalt, with his sword prepar'd; + Which, as he breath'd defiance to my ears, + He swung about his head and cut the winds, + Who, nothing hurt withal, hiss'd him in scorn. + While we were interchanging thrusts and blows, + Came more and more, and fought on part and part, + Till the Prince came, who parted either part. + + M. Wife. O, where is Romeo? Saw you him to-day? + Right glad I am he was not at this fray. + + Ben. Madam, an hour before the worshipp'd sun + Peer'd forth the golden window of the East, + A troubled mind drave me to walk abroad; + Where, underneath the grove of sycamore + That westward rooteth from the city's side, + So early walking did I see your son. + Towards him I made; but he was ware of me + And stole into the covert of the wood. + I- measuring his affections by my own, + Which then most sought where most might not be found, + Being one too many by my weary self- + Pursu'd my humour, not Pursuing his, + And gladly shunn'd who gladly fled from me. + + Mon. Many a morning hath he there been seen, + With tears augmenting the fresh morning's dew, + Adding to clouds more clouds with his deep sighs; + But all so soon as the all-cheering sun + Should in the furthest East bean to draw + The shady curtains from Aurora's bed, + Away from light steals home my heavy son + And private in his chamber pens himself, + Shuts up his windows, locks fair daylight + And makes himself an artificial night. + Black and portentous must this humour prove + Unless good counsel may the cause remove. + + Ben. My noble uncle, do you know the cause? + + Mon. I neither know it nor can learn of him + + Ben. Have you importun'd him by any means? + + Mon. Both by myself and many other friend; + But he, his own affections' counsellor, + Is to himself- I will not say how true- + But to himself so secret and so close, + So far from sounding and discovery, + As is the bud bit with an envious worm + Ere he can spread his sweet leaves to the air + Or dedicate his beauty to the sun. + Could we but learn from whence his sorrows grow, + We would as willingly give cure as know. + + Enter Romeo. + + + Ben. See, where he comes. So please you step aside, + I'll know his grievance, or be much denied. + + Mon. I would thou wert so happy by thy stay + To hear true shrift. Come, madam, let's away, + Exeunt [Montague and Wife]. + + Ben. Good morrow, cousin. + + Rom. Is the day so young? + + Ben. But new struck nine. + + Rom. Ay me! sad hours seem long. + Was that my father that went hence so fast? + + Ben. It was. What sadness lengthens Romeo's hours? + + Rom. Not having that which having makes them short. + + Ben. In love? + + Rom. Out- + + Ben. Of love? + + Rom. Out of her favour where I am in love. + + Ben. Alas that love, so gentle in his view, + Should be so tyrannous and rough in proof! + + Rom. Alas that love, whose view is muffled still, + Should without eyes see pathways to his will! + Where shall we dine? O me! What fray was here? + Yet tell me not, for I have heard it all. + Here's much to do with hate, but more with love. + Why then, O brawling love! O loving hate! + O anything, of nothing first create! + O heavy lightness! serious vanity! + Misshapen chaos of well-seeming forms! + Feather of lead, bright smoke, cold fire, sick health! + Still-waking sleep, that is not what it is + This love feel I, that feel no love in this. + Dost thou not laugh? + + Ben. No, coz, I rather weep. + + Rom. Good heart, at what? + + Ben. At thy good heart's oppression. + + Rom. Why, such is love's transgression. + Griefs of mine own lie heavy in my breast, + Which thou wilt propagate, to have it prest + With more of thine. This love that thou hast shown + Doth add more grief to too much of mine own. + Love is a smoke rais'd with the fume of sighs; + Being purg'd, a fire sparkling in lovers' eyes; + Being vex'd, a sea nourish'd with lovers' tears. + What is it else? A madness most discreet, + A choking gall, and a preserving sweet. + Farewell, my coz. + + Ben. Soft! I will go along. + An if you leave me so, you do me wrong. + + Rom. Tut! I have lost myself; I am not here: + This is not Romeo, he's some other where. + + Ben. Tell me in sadness, who is that you love? + + Rom. What, shall I groan and tell thee? + + Ben. Groan? Why, no; + But sadly tell me who. + + Rom. Bid a sick man in sadness make his will. + Ah, word ill urg'd to one that is so ill! + In sadness, cousin, I do love a woman. + + Ben. I aim'd so near when I suppos'd you lov'd. + + Rom. A right good markman! And she's fair I love. + + Ben. A right fair mark, fair coz, is soonest hit. + + Rom. Well, in that hit you miss. She'll not be hit + With Cupid's arrow. She hath Dian's wit, + And, in strong proof of chastity well arm'd, + From Love's weak childish bow she lives unharm'd. + She will not stay the siege of loving terms, + Nor bide th' encounter of assailing eyes, + Nor ope her lap to saint-seducing gold. + O, she's rich in beauty; only poor + That, when she dies, with beauty dies her store. + + Ben. Then she hath sworn that she will still live chaste? + + Rom. She hath, and in that sparing makes huge waste; + For beauty, starv'd with her severity, + Cuts beauty off from all posterity. + She is too fair, too wise, wisely too fair, + To merit bliss by making me despair. + She hath forsworn to love, and in that vow + Do I live dead that live to tell it now. + + Ben. Be rul'd by me: forget to think of her. + + Rom. O, teach me how I should forget to think! + + Ben. By giving liberty unto thine eyes. + Examine other beauties. + + Rom. 'Tis the way + To call hers (exquisite) in question more. + These happy masks that kiss fair ladies' brows, + Being black puts us in mind they hide the fair. + He that is strucken blind cannot forget + The precious treasure of his eyesight lost. + Show me a mistress that is passing fair, + What doth her beauty serve but as a note + Where I may read who pass'd that passing fair? + Farewell. Thou canst not teach me to forget. + + Ben. I'll pay that doctrine, or else die in debt. Exeunt. + + + + +Scene II. +A Street. + +Enter Capulet, County Paris, and [Servant] -the Clown. + + + Cap. But Montague is bound as well as I, + In penalty alike; and 'tis not hard, I think, + For men so old as we to keep the peace. + + Par. Of honourable reckoning are you both, + And pity 'tis you liv'd at odds so long. + But now, my lord, what say you to my suit? + + Cap. But saying o'er what I have said before: + My child is yet a stranger in the world, + She hath not seen the change of fourteen years; + Let two more summers wither in their pride + Ere we may think her ripe to be a bride. + + Par. Younger than she are happy mothers made. + + Cap. And too soon marr'd are those so early made. + The earth hath swallowed all my hopes but she; + She is the hopeful lady of my earth. + But woo her, gentle Paris, get her heart; + My will to her consent is but a part. + An she agree, within her scope of choice + Lies my consent and fair according voice. + This night I hold an old accustom'd feast, + Whereto I have invited many a guest, + Such as I love; and you among the store, + One more, most welcome, makes my number more. + At my poor house look to behold this night + Earth-treading stars that make dark heaven light. + Such comfort as do lusty young men feel + When well apparell'd April on the heel + Of limping Winter treads, even such delight + Among fresh female buds shall you this night + Inherit at my house. Hear all, all see, + And like her most whose merit most shall be; + Which, on more view of many, mine, being one, + May stand in number, though in reck'ning none. + Come, go with me. [To Servant, giving him a paper] Go, + sirrah, trudge about + Through fair Verona; find those persons out + Whose names are written there, and to them say, + My house and welcome on their pleasure stay- + Exeunt [Capulet and Paris]. + + Serv. Find them out whose names are written here? It is written + that the shoemaker should meddle with his yard and the tailor + with his last, the fisher with his pencil and the painter + with his nets; but I am sent to find those persons whose names are + here writ, and can never find what names the writing person + hath here writ. I must to the learned. In good time! + + Enter Benvolio and Romeo. + + + Ben. Tut, man, one fire burns out another's burning; + One pain is lessoned by another's anguish; + Turn giddy, and be holp by backward turning; + One desperate grief cures with another's languish. + Take thou some new infection to thy eye, + And the rank poison of the old will die. + + Rom. Your plantain leaf is excellent for that. + + Ben. For what, I pray thee? + + Rom. For your broken shin. + + Ben. Why, Romeo, art thou mad? + + Rom. Not mad, but bound more than a madman is; + Shut up in Prison, kept without my food, + Whipp'd and tormented and- God-den, good fellow. + + Serv. God gi' go-den. I pray, sir, can you read? + + Rom. Ay, mine own fortune in my misery. + + Serv. Perhaps you have learned it without book. But I pray, can + you read anything you see? + + Rom. Ay, If I know the letters and the language. + + Serv. Ye say honestly. Rest you merry! + + Rom. Stay, fellow; I can read. He reads. + + 'Signior Martino and his wife and daughters; + County Anselmo and his beauteous sisters; + The lady widow of Vitruvio; + Signior Placentio and His lovely nieces; + Mercutio and his brother Valentine; + Mine uncle Capulet, his wife, and daughters; + My fair niece Rosaline and Livia; + Signior Valentio and His cousin Tybalt; + Lucio and the lively Helena.' + + [Gives back the paper.] A fair assembly. Whither should they + come? + + Serv. Up. + + Rom. Whither? + + Serv. To supper, to our house. + + Rom. Whose house? + + Serv. My master's. + + Rom. Indeed I should have ask'd you that before. + + Serv. Now I'll tell you without asking. My master is the great + rich Capulet; and if you be not of the house of Montagues, I pray + come and crush a cup of wine. Rest you merry! Exit. + + Ben. At this same ancient feast of Capulet's + Sups the fair Rosaline whom thou so lov'st; + With all the admired beauties of Verona. + Go thither, and with unattainted eye + Compare her face with some that I shall show, + And I will make thee think thy swan a crow. + + Rom. When the devout religion of mine eye + Maintains such falsehood, then turn tears to fires; + And these, who, often drown'd, could never die, + Transparent heretics, be burnt for liars! + One fairer than my love? The all-seeing sun + Ne'er saw her match since first the world begun. + + Ben. Tut! you saw her fair, none else being by, + Herself pois'd with herself in either eye; + But in that crystal scales let there be weigh'd + Your lady's love against some other maid + That I will show you shining at this feast, + And she shall scant show well that now seems best. + + Rom. I'll go along, no such sight to be shown, + But to rejoice in splendour of my own. [Exeunt.] + + + + +Scene III. +Capulet's house. + +Enter Capulet's Wife, and Nurse. + + + Wife. Nurse, where's my daughter? Call her forth to me. + + Nurse. Now, by my maidenhead at twelve year old, + I bade her come. What, lamb! what ladybird! + God forbid! Where's this girl? What, Juliet! + + Enter Juliet. + + + Jul. How now? Who calls? + + Nurse. Your mother. + + Jul. Madam, I am here. + What is your will? + + Wife. This is the matter- Nurse, give leave awhile, + We must talk in secret. Nurse, come back again; + I have rememb'red me, thou's hear our counsel. + Thou knowest my daughter's of a pretty age. + + Nurse. Faith, I can tell her age unto an hour. + + Wife. She's not fourteen. + + Nurse. I'll lay fourteen of my teeth- + And yet, to my teen be it spoken, I have but four- + She is not fourteen. How long is it now + To Lammastide? + + Wife. A fortnight and odd days. + + Nurse. Even or odd, of all days in the year, + Come Lammas Eve at night shall she be fourteen. + Susan and she (God rest all Christian souls!) + Were of an age. Well, Susan is with God; + She was too good for me. But, as I said, + On Lammas Eve at night shall she be fourteen; + That shall she, marry; I remember it well. + 'Tis since the earthquake now eleven years; + And she was wean'd (I never shall forget it), + Of all the days of the year, upon that day; + For I had then laid wormwood to my dug, + Sitting in the sun under the dovehouse wall. + My lord and you were then at Mantua. + Nay, I do bear a brain. But, as I said, + When it did taste the wormwood on the nipple + Of my dug and felt it bitter, pretty fool, + To see it tetchy and fall out with the dug! + Shake, quoth the dovehouse! 'Twas no need, I trow, + To bid me trudge. + And since that time it is eleven years, + For then she could stand high-lone; nay, by th' rood, + She could have run and waddled all about; + For even the day before, she broke her brow; + And then my husband (God be with his soul! + 'A was a merry man) took up the child. + 'Yea,' quoth he, 'dost thou fall upon thy face? + Thou wilt fall backward when thou hast more wit; + Wilt thou not, Jule?' and, by my holidam, + The pretty wretch left crying, and said 'Ay.' + To see now how a jest shall come about! + I warrant, an I should live a thousand yeas, + I never should forget it. 'Wilt thou not, Jule?' quoth he, + And, pretty fool, it stinted, and said 'Ay.' + + Wife. Enough of this. I pray thee hold thy peace. + + Nurse. Yes, madam. Yet I cannot choose but laugh + To think it should leave crying and say 'Ay.' + And yet, I warrant, it bad upon it brow + A bump as big as a young cock'rel's stone; + A perilous knock; and it cried bitterly. + 'Yea,' quoth my husband, 'fall'st upon thy face? + Thou wilt fall backward when thou comest to age; + Wilt thou not, Jule?' It stinted, and said 'Ay.' + + Jul. And stint thou too, I pray thee, nurse, say I. + + Nurse. Peace, I have done. God mark thee to his grace! + Thou wast the prettiest babe that e'er I nurs'd. + An I might live to see thee married once, I have my wish. + + Wife. Marry, that 'marry' is the very theme + I came to talk of. Tell me, daughter Juliet, + How stands your disposition to be married? + + Jul. It is an honour that I dream not of. + + Nurse. An honour? Were not I thine only nurse, + I would say thou hadst suck'd wisdom from thy teat. + + Wife. Well, think of marriage now. Younger than you, + Here in Verona, ladies of esteem, + Are made already mothers. By my count, + I was your mother much upon these years + That you are now a maid. Thus then in brief: + The valiant Paris seeks you for his love. + + Nurse. A man, young lady! lady, such a man + As all the world- why he's a man of wax. + + Wife. Verona's summer hath not such a flower. + + Nurse. Nay, he's a flower, in faith- a very flower. + + Wife. What say you? Can you love the gentleman? + This night you shall behold him at our feast. + Read o'er the volume of young Paris' face, + And find delight writ there with beauty's pen; + Examine every married lineament, + And see how one another lends content; + And what obscur'd in this fair volume lies + Find written in the margent of his eyes, + This precious book of love, this unbound lover, + To beautify him only lacks a cover. + The fish lives in the sea, and 'tis much pride + For fair without the fair within to hide. + That book in many's eyes doth share the glory, + That in gold clasps locks in the golden story; + So shall you share all that he doth possess, + By having him making yourself no less. + + Nurse. No less? Nay, bigger! Women grow by men + + Wife. Speak briefly, can you like of Paris' love? + + Jul. I'll look to like, if looking liking move; + But no more deep will I endart mine eye + Than your consent gives strength to make it fly. + + Enter Servingman. + + + Serv. Madam, the guests are come, supper serv'd up, you call'd, + my young lady ask'd for, the nurse curs'd in the pantry, and + everything in extremity. I must hence to wait. I beseech you + follow straight. + + Wife. We follow thee. Exit [Servingman]. + Juliet, the County stays. + + Nurse. Go, girl, seek happy nights to happy days. + Exeunt. + + + + +Scene IV. +A street. + +Enter Romeo, Mercutio, Benvolio, with five or six other Maskers; +Torchbearers. + + + Rom. What, shall this speech be spoke for our excuse? + Or shall we on without apology? + + Ben. The date is out of such prolixity. + We'll have no Cupid hoodwink'd with a scarf, + Bearing a Tartar's painted bow of lath, + Scaring the ladies like a crowkeeper; + Nor no without-book prologue, faintly spoke + After the prompter, for our entrance; + But, let them measure us by what they will, + We'll measure them a measure, and be gone. + + Rom. Give me a torch. I am not for this ambling. + Being but heavy, I will bear the light. + + Mer. Nay, gentle Romeo, we must have you dance. + + Rom. Not I, believe me. You have dancing shoes + With nimble soles; I have a soul of lead + So stakes me to the ground I cannot move. + + Mer. You are a lover. Borrow Cupid's wings + And soar with them above a common bound. + + Rom. I am too sore enpierced with his shaft + To soar with his light feathers; and so bound + I cannot bound a pitch above dull woe. + Under love's heavy burthen do I sink. + + Mer. And, to sink in it, should you burthen love- + Too great oppression for a tender thing. + + Rom. Is love a tender thing? It is too rough, + Too rude, too boist'rous, and it pricks like thorn. + + Mer. If love be rough with you, be rough with love. + Prick love for pricking, and you beat love down. + Give me a case to put my visage in. + A visor for a visor! What care I + What curious eye doth quote deformities? + Here are the beetle brows shall blush for me. + + Ben. Come, knock and enter; and no sooner in + But every man betake him to his legs. + + Rom. A torch for me! Let wantons light of heart + Tickle the senseless rushes with their heels; + For I am proverb'd with a grandsire phrase, + I'll be a candle-holder and look on; + The game was ne'er so fair, and I am done. + + Mer. Tut! dun's the mouse, the constable's own word! + If thou art Dun, we'll draw thee from the mire + Of this sir-reverence love, wherein thou stick'st + Up to the ears. Come, we burn daylight, ho! + + Rom. Nay, that's not so. + + Mer. I mean, sir, in delay + We waste our lights in vain, like lamps by day. + Take our good meaning, for our judgment sits + Five times in that ere once in our five wits. + + Rom. And we mean well, in going to this masque; + But 'tis no wit to go. + + Mer. Why, may one ask? + + Rom. I dreamt a dream to-night. + + Mer. And so did I. + + Rom. Well, what was yours? + + Mer. That dreamers often lie. + + Rom. In bed asleep, while they do dream things true. + + Mer. O, then I see Queen Mab hath been with you. + She is the fairies' midwife, and she comes + In shape no bigger than an agate stone + On the forefinger of an alderman, + Drawn with a team of little atomies + Athwart men's noses as they lie asleep; + Her wagon spokes made of long spinners' legs, + The cover, of the wings of grasshoppers; + Her traces, of the smallest spider's web; + Her collars, of the moonshine's wat'ry beams; + Her whip, of cricket's bone; the lash, of film; + Her wagoner, a small grey-coated gnat, + Not half so big as a round little worm + Prick'd from the lazy finger of a maid; + Her chariot is an empty hazelnut, + Made by the joiner squirrel or old grub, + Time out o' mind the fairies' coachmakers. + And in this state she 'gallops night by night + Through lovers' brains, and then they dream of love; + O'er courtiers' knees, that dream on cursies straight; + O'er lawyers' fingers, who straight dream on fees; + O'er ladies' lips, who straight on kisses dream, + Which oft the angry Mab with blisters plagues, + Because their breaths with sweetmeats tainted are. + Sometime she gallops o'er a courtier's nose, + And then dreams he of smelling out a suit; + And sometime comes she with a tithe-pig's tail + Tickling a parson's nose as 'a lies asleep, + Then dreams he of another benefice. + Sometimes she driveth o'er a soldier's neck, + And then dreams he of cutting foreign throats, + Of breaches, ambuscadoes, Spanish blades, + Of healths five fadom deep; and then anon + Drums in his ear, at which he starts and wakes, + And being thus frighted, swears a prayer or two + And sleeps again. This is that very Mab + That plats the manes of horses in the night + And bakes the elflocks in foul sluttish, hairs, + Which once untangled much misfortune bodes + This is the hag, when maids lie on their backs, + That presses them and learns them first to bear, + Making them women of good carriage. + This is she- + + Rom. Peace, peace, Mercutio, peace! + Thou talk'st of nothing. + + Mer. True, I talk of dreams; + Which are the children of an idle brain, + Begot of nothing but vain fantasy; + Which is as thin of substance as the air, + And more inconstant than the wind, who wooes + Even now the frozen bosom of the North + And, being anger'd, puffs away from thence, + Turning his face to the dew-dropping South. + + Ben. This wind you talk of blows us from ourselves. + Supper is done, and we shall come too late. + + Rom. I fear, too early; for my mind misgives + Some consequence, yet hanging in the stars, + Shall bitterly begin his fearful date + With this night's revels and expire the term + Of a despised life, clos'd in my breast, + By some vile forfeit of untimely death. + But he that hath the steerage of my course + Direct my sail! On, lusty gentlemen! + + Ben. Strike, drum. + They march about the stage. [Exeunt.] + + + + +Scene V. +Capulet's house. + +Servingmen come forth with napkins. + + 1. Serv. Where's Potpan, that he helps not to take away? + He shift a trencher! he scrape a trencher! + 2. Serv. When good manners shall lie all in one or two men's + hands, and they unwash'd too, 'tis a foul thing. + 1. Serv. Away with the join-stools, remove the court-cubbert, + look to the plate. Good thou, save me a piece of marchpane and, as + thou loves me, let the porter let in Susan Grindstone and +Nell. + Anthony, and Potpan! + 2. Serv. Ay, boy, ready. + 1. Serv. You are look'd for and call'd for, ask'd for and + sought for, in the great chamber. + 3. Serv. We cannot be here and there too. Cheerly, boys! + Be brisk awhile, and the longer liver take all. Exeunt. + + Enter the Maskers, Enter, [with Servants,] Capulet, his Wife, + Juliet, Tybalt, and all the Guests + and Gentlewomen to the Maskers. + + + Cap. Welcome, gentlemen! Ladies that have their toes + Unplagu'd with corns will have a bout with you. + Ah ha, my mistresses! which of you all + Will now deny to dance? She that makes dainty, + She I'll swear hath corns. Am I come near ye now? + Welcome, gentlemen! I have seen the day + That I have worn a visor and could tell + A whispering tale in a fair lady's ear, + Such as would please. 'Tis gone, 'tis gone, 'tis gone! + You are welcome, gentlemen! Come, musicians, play. + A hall, a hall! give room! and foot it, girls. + Music plays, and they dance. + More light, you knaves! and turn the tables up, + And quench the fire, the room is grown too hot. + Ah, sirrah, this unlook'd-for sport comes well. + Nay, sit, nay, sit, good cousin Capulet, + For you and I are past our dancing days. + How long is't now since last yourself and I + Were in a mask? + 2. Cap. By'r Lady, thirty years. + + Cap. What, man? 'Tis not so much, 'tis not so much! + 'Tis since the nuptial of Lucentio, + Come Pentecost as quickly as it will, + Some five-and-twenty years, and then we mask'd. + 2. Cap. 'Tis more, 'tis more! His son is elder, sir; + His son is thirty. + + Cap. Will you tell me that? + His son was but a ward two years ago. + + Rom. [to a Servingman] What lady's that, which doth enrich the + hand Of yonder knight? + + Serv. I know not, sir. + + Rom. O, she doth teach the torches to burn bright! + It seems she hangs upon the cheek of night + Like a rich jewel in an Ethiop's ear- + Beauty too rich for use, for earth too dear! + So shows a snowy dove trooping with crows + As yonder lady o'er her fellows shows. + The measure done, I'll watch her place of stand + And, touching hers, make blessed my rude hand. + Did my heart love till now? Forswear it, sight! + For I ne'er saw true beauty till this night. + + Tyb. This, by his voice, should be a Montague. + Fetch me my rapier, boy. What, dares the slave + Come hither, cover'd with an antic face, + To fleer and scorn at our solemnity? + Now, by the stock and honour of my kin, + To strike him dead I hold it not a sin. + + Cap. Why, how now, kinsman? Wherefore storm you so? + + Tyb. Uncle, this is a Montague, our foe; + A villain, that is hither come in spite + To scorn at our solemnity this night. + + Cap. Young Romeo is it? + + Tyb. 'Tis he, that villain Romeo. + + Cap. Content thee, gentle coz, let him alone. + 'A bears him like a portly gentleman, + And, to say truth, Verona brags of him + To be a virtuous and well-govern'd youth. + I would not for the wealth of all this town + Here in my house do him disparagement. + Therefore be patient, take no note of him. + It is my will; the which if thou respect, + Show a fair presence and put off these frowns, + An ill-beseeming semblance for a feast. + + Tyb. It fits when such a villain is a guest. + I'll not endure him. + + Cap. He shall be endur'd. + What, goodman boy? I say he shall. Go to! + Am I the master here, or you? Go to! + You'll not endure him? God shall mend my soul! + You'll make a mutiny among my guests! + You will set cock-a-hoop! you'll be the man! + + Tyb. Why, uncle, 'tis a shame. + + Cap. Go to, go to! + You are a saucy boy. Is't so, indeed? + This trick may chance to scathe you. I know what. + You must contrary me! Marry, 'tis time.- + Well said, my hearts!- You are a princox- go! + Be quiet, or- More light, more light!- For shame! + I'll make you quiet; what!- Cheerly, my hearts! + + Tyb. Patience perforce with wilful choler meeting + Makes my flesh tremble in their different greeting. + I will withdraw; but this intrusion shall, + Now seeming sweet, convert to bitt'rest gall. Exit. + + Rom. If I profane with my unworthiest hand + This holy shrine, the gentle fine is this: + My lips, two blushing pilgrims, ready stand + To smooth that rough touch with a tender kiss. + + Jul. Good pilgrim, you do wrong your hand too much, + Which mannerly devotion shows in this; + For saints have hands that pilgrims' hands do touch, + And palm to palm is holy palmers' kiss. + + Rom. Have not saints lips, and holy palmers too? + + Jul. Ay, pilgrim, lips that they must use in pray'r. + + Rom. O, then, dear saint, let lips do what hands do! + They pray; grant thou, lest faith turn to despair. + + Jul. Saints do not move, though grant for prayers' sake. + + Rom. Then move not while my prayer's effect I take. + Thus from my lips, by thine my sin is purg'd. [Kisses her.] + + Jul. Then have my lips the sin that they have took. + + Rom. Sin from my lips? O trespass sweetly urg'd! + Give me my sin again. [Kisses her.] + + Jul. You kiss by th' book. + + Nurse. Madam, your mother craves a word with you. + + Rom. What is her mother? + + Nurse. Marry, bachelor, + Her mother is the lady of the house. + And a good lady, and a wise and virtuous. + I nurs'd her daughter that you talk'd withal. + I tell you, he that can lay hold of her + Shall have the chinks. + + Rom. Is she a Capulet? + O dear account! my life is my foe's debt. + + Ben. Away, be gone; the sport is at the best. + + Rom. Ay, so I fear; the more is my unrest. + + Cap. Nay, gentlemen, prepare not to be gone; + We have a trifling foolish banquet towards. + Is it e'en so? Why then, I thank you all. + I thank you, honest gentlemen. Good night. + More torches here! [Exeunt Maskers.] Come on then, let's to bed. + Ah, sirrah, by my fay, it waxes late; + I'll to my rest. + Exeunt [all but Juliet and Nurse]. + + Jul. Come hither, nurse. What is yond gentleman? + + Nurse. The son and heir of old Tiberio. + + Jul. What's he that now is going out of door? + + Nurse. Marry, that, I think, be young Petruchio. + + Jul. What's he that follows there, that would not dance? + + Nurse. I know not. + + Jul. Go ask his name.- If he be married, + My grave is like to be my wedding bed. + + Nurse. His name is Romeo, and a Montague, + The only son of your great enemy. + + Jul. My only love, sprung from my only hate! + Too early seen unknown, and known too late! + Prodigious birth of love it is to me + That I must love a loathed enemy. + + Nurse. What's this? what's this? + + Jul. A rhyme I learnt even now + Of one I danc'd withal. + One calls within, 'Juliet.' + + Nurse. Anon, anon! + Come, let's away; the strangers all are gone. Exeunt. + + + + +PROLOGUE + +Enter Chorus. + + + Chor. Now old desire doth in his deathbed lie, + And young affection gapes to be his heir; + That fair for which love groan'd for and would die, + With tender Juliet match'd, is now not fair. + Now Romeo is belov'd, and loves again, + Alike bewitched by the charm of looks; + But to his foe suppos'd he must complain, + And she steal love's sweet bait from fearful hooks. + Being held a foe, he may not have access + To breathe such vows as lovers use to swear, + And she as much in love, her means much less + To meet her new beloved anywhere; + But passion lends them power, time means, to meet, + Temp'ring extremities with extreme sweet. +Exit. + + + + +ACT II. Scene I. +A lane by the wall of Capulet's orchard. + +Enter Romeo alone. + + + Rom. Can I go forward when my heart is here? + Turn back, dull earth, and find thy centre out. + [Climbs the wall and leaps down within it.] + + Enter Benvolio with Mercutio. + + + Ben. Romeo! my cousin Romeo! Romeo! + + Mer. He is wise, + And, on my life, hath stol'n him home to bed. + + Ben. He ran this way, and leapt this orchard wall. + Call, good Mercutio. + + Mer. Nay, I'll conjure too. + Romeo! humours! madman! passion! lover! + Appear thou in the likeness of a sigh; + Speak but one rhyme, and I am satisfied! + Cry but 'Ay me!' pronounce but 'love' and 'dove'; + Speak to my gossip Venus one fair word, + One nickname for her purblind son and heir, + Young Adam Cupid, he that shot so trim + When King Cophetua lov'd the beggar maid! + He heareth not, he stirreth not, be moveth not; + The ape is dead, and I must conjure him. + I conjure thee by Rosaline's bright eyes. + By her high forehead and her scarlet lip, + By her fine foot, straight leg, and quivering thigh, + And the demesnes that there adjacent lie, + That in thy likeness thou appear to us! + + Ben. An if he hear thee, thou wilt anger him. + + Mer. This cannot anger him. 'Twould anger him + To raise a spirit in his mistress' circle + Of some strange nature, letting it there stand + Till she had laid it and conjur'd it down. + That were some spite; my invocation + Is fair and honest: in his mistress' name, + I conjure only but to raise up him. + + Ben. Come, he hath hid himself among these trees + To be consorted with the humorous night. + Blind is his love and best befits the dark. + + Mer. If love be blind, love cannot hit the mark. + Now will he sit under a medlar tree + And wish his mistress were that kind of fruit + As maids call medlars when they laugh alone. + O, Romeo, that she were, O that she were + An open et cetera, thou a pop'rin pear! + Romeo, good night. I'll to my truckle-bed; + This field-bed is too cold for me to sleep. + Come, shall we go? + + Ben. Go then, for 'tis in vain + 'To seek him here that means not to be found. + Exeunt. + + + + +Scene II. +Capulet's orchard. + +Enter Romeo. + + + Rom. He jests at scars that never felt a wound. + + Enter Juliet above at a window. + + But soft! What light through yonder window breaks? + It is the East, and Juliet is the sun! + Arise, fair sun, and kill the envious moon, + Who is already sick and pale with grief + That thou her maid art far more fair than she. + Be not her maid, since she is envious. + Her vestal livery is but sick and green, + And none but fools do wear it. Cast it off. + It is my lady; O, it is my love! + O that she knew she were! + She speaks, yet she says nothing. What of that? + Her eye discourses; I will answer it. + I am too bold; 'tis not to me she speaks. + Two of the fairest stars in all the heaven, + Having some business, do entreat her eyes + To twinkle in their spheres till they return. + What if her eyes were there, they in her head? + The brightness of her cheek would shame those stars + As daylight doth a lamp; her eyes in heaven + Would through the airy region stream so bright + That birds would sing and think it were not night. + See how she leans her cheek upon her hand! + O that I were a glove upon that hand, + That I might touch that cheek! + + Jul. Ay me! + + Rom. She speaks. + O, speak again, bright angel! for thou art + As glorious to this night, being o'er my head, + As is a winged messenger of heaven + Unto the white-upturned wond'ring eyes + Of mortals that fall back to gaze on him + When he bestrides the lazy-pacing clouds + And sails upon the bosom of the air. + + Jul. O Romeo, Romeo! wherefore art thou Romeo? + Deny thy father and refuse thy name! + Or, if thou wilt not, be but sworn my love, + And I'll no longer be a Capulet. + + Rom. [aside] Shall I hear more, or shall I speak at this? + + Jul. 'Tis but thy name that is my enemy. + Thou art thyself, though not a Montague. + What's Montague? it is nor hand, nor foot, + Nor arm, nor face, nor any other part + Belonging to a man. O, be some other name! + What's in a name? That which we call a rose + By any other name would smell as sweet. + So Romeo would, were he not Romeo call'd, + Retain that dear perfection which he owes + Without that title. Romeo, doff thy name; + And for that name, which is no part of thee, + Take all myself. + + Rom. I take thee at thy word. + Call me but love, and I'll be new baptiz'd; + Henceforth I never will be Romeo. + + Jul. What man art thou that, thus bescreen'd in night, + So stumblest on my counsel? + + Rom. By a name + I know not how to tell thee who I am. + My name, dear saint, is hateful to myself, + Because it is an enemy to thee. + Had I it written, I would tear the word. + + Jul. My ears have yet not drunk a hundred words + Of that tongue's utterance, yet I know the sound. + Art thou not Romeo, and a Montague? + + Rom. Neither, fair saint, if either thee dislike. + + Jul. How cam'st thou hither, tell me, and wherefore? + The orchard walls are high and hard to climb, + And the place death, considering who thou art, + If any of my kinsmen find thee here. + + Rom. With love's light wings did I o'erperch these walls; + For stony limits cannot hold love out, + And what love can do, that dares love attempt. + Therefore thy kinsmen are no let to me. + + Jul. If they do see thee, they will murther thee. + + Rom. Alack, there lies more peril in thine eye + Than twenty of their swords! Look thou but sweet, + And I am proof against their enmity. + + Jul. I would not for the world they saw thee here. + + Rom. I have night's cloak to hide me from their sight; + And but thou love me, let them find me here. + My life were better ended by their hate + Than death prorogued, wanting of thy love. + + Jul. By whose direction found'st thou out this place? + + Rom. By love, that first did prompt me to enquire. + He lent me counsel, and I lent him eyes. + I am no pilot; yet, wert thou as far + As that vast shore wash'd with the farthest sea, + I would adventure for such merchandise. + + Jul. Thou knowest the mask of night is on my face; + Else would a maiden blush bepaint my cheek + For that which thou hast heard me speak to-night. + Fain would I dwell on form- fain, fain deny + What I have spoke; but farewell compliment! + Dost thou love me, I know thou wilt say 'Ay'; + And I will take thy word. Yet, if thou swear'st, + Thou mayst prove false. At lovers' perjuries, + They say Jove laughs. O gentle Romeo, + If thou dost love, pronounce it faithfully. + Or if thou thinkest I am too quickly won, + I'll frown, and be perverse, and say thee nay, + So thou wilt woo; but else, not for the world. + In truth, fair Montague, I am too fond, + And therefore thou mayst think my haviour light; + But trust me, gentleman, I'll prove more true + Than those that have more cunning to be strange. + I should have been more strange, I must confess, + But that thou overheard'st, ere I was ware, + My true-love passion. Therefore pardon me, + And not impute this yielding to light love, + Which the dark night hath so discovered. + + Rom. Lady, by yonder blessed moon I swear, + That tips with silver all these fruit-tree tops- + + Jul. O, swear not by the moon, th' inconstant moon, + That monthly changes in her circled orb, + Lest that thy love prove likewise variable. + + Rom. What shall I swear by? + + Jul. Do not swear at all; + Or if thou wilt, swear by thy gracious self, + Which is the god of my idolatry, + And I'll believe thee. + + Rom. If my heart's dear love- + + Jul. Well, do not swear. Although I joy in thee, + I have no joy of this contract to-night. + It is too rash, too unadvis'd, too sudden; + Too like the lightning, which doth cease to be + Ere one can say 'It lightens.' Sweet, good night! + This bud of love, by summer's ripening breath, + May prove a beauteous flow'r when next we meet. + Good night, good night! As sweet repose and rest + Come to thy heart as that within my breast! + + Rom. O, wilt thou leave me so unsatisfied? + + Jul. What satisfaction canst thou have to-night? + + Rom. Th' exchange of thy love's faithful vow for mine. + + Jul. I gave thee mine before thou didst request it; + And yet I would it were to give again. + + Rom. Would'st thou withdraw it? For what purpose, love? + + Jul. But to be frank and give it thee again. + And yet I wish but for the thing I have. + My bounty is as boundless as the sea, + My love as deep; the more I give to thee, + The more I have, for both are infinite. + I hear some noise within. Dear love, adieu! + [Nurse] calls within. + Anon, good nurse! Sweet Montague, be true. + Stay but a little, I will come again. [Exit.] + + Rom. O blessed, blessed night! I am afeard, + Being in night, all this is but a dream, + Too flattering-sweet to be substantial. + + Enter Juliet above. + + + Jul. Three words, dear Romeo, and good night indeed. + If that thy bent of love be honourable, + Thy purpose marriage, send me word to-morrow, + By one that I'll procure to come to thee, + Where and what time thou wilt perform the rite; + And all my fortunes at thy foot I'll lay + And follow thee my lord throughout the world. + + Nurse. (within) Madam! + + Jul. I come, anon.- But if thou meanest not well, + I do beseech thee- + + Nurse. (within) Madam! + + Jul. By-and-by I come.- + To cease thy suit and leave me to my grief. + To-morrow will I send. + + Rom. So thrive my soul- + + Jul. A thousand times good night! Exit. + + Rom. A thousand times the worse, to want thy light! + Love goes toward love as schoolboys from their books; + But love from love, towards school with heavy looks. + + Enter Juliet again, [above]. + + + Jul. Hist! Romeo, hist! O for a falconer's voice + To lure this tassel-gentle back again! + Bondage is hoarse and may not speak aloud; + Else would I tear the cave where Echo lies, + And make her airy tongue more hoarse than mine + With repetition of my Romeo's name. + Romeo! + + Rom. It is my soul that calls upon my name. + How silver-sweet sound lovers' tongues by night, + Like softest music to attending ears! + + Jul. Romeo! + + Rom. My dear? + + Jul. At what o'clock to-morrow + Shall I send to thee? + + Rom. By the hour of nine. + + Jul. I will not fail. 'Tis twenty years till then. + I have forgot why I did call thee back. + + Rom. Let me stand here till thou remember it. + + Jul. I shall forget, to have thee still stand there, + Rememb'ring how I love thy company. + + Rom. And I'll still stay, to have thee still forget, + Forgetting any other home but this. + + Jul. 'Tis almost morning. I would have thee gone- + And yet no farther than a wanton's bird, + That lets it hop a little from her hand, + Like a poor prisoner in his twisted gyves, + And with a silk thread plucks it back again, + So loving-jealous of his liberty. + + Rom. I would I were thy bird. + + Jul. Sweet, so would I. + Yet I should kill thee with much cherishing. + Good night, good night! Parting is such sweet sorrow, + That I shall say good night till it be morrow. + [Exit.] + + Rom. Sleep dwell upon thine eyes, peace in thy breast! + Would I were sleep and peace, so sweet to rest! + Hence will I to my ghostly father's cell, + His help to crave and my dear hap to tell. + Exit + + + + +Scene III. +Friar Laurence's cell. + +Enter Friar, [Laurence] alone, with a basket. + + + Friar. The grey-ey'd morn smiles on the frowning night, + Check'ring the Eastern clouds with streaks of light; + And flecked darkness like a drunkard reels + From forth day's path and Titan's fiery wheels. + Non, ere the sun advance his burning eye + The day to cheer and night's dank dew to dry, + I must up-fill this osier cage of ours + With baleful weeds and precious-juiced flowers. + The earth that's nature's mother is her tomb. + What is her burying gave, that is her womb; + And from her womb children of divers kind + We sucking on her natural bosom find; + Many for many virtues excellent, + None but for some, and yet all different. + O, mickle is the powerful grace that lies + In plants, herbs, stones, and their true qualities; + For naught so vile that on the earth doth live + But to the earth some special good doth give; + Nor aught so good but, strain'd from that fair use, + Revolts from true birth, stumbling on abuse. + Virtue itself turns vice, being misapplied, + And vice sometime's by action dignified. + Within the infant rind of this small flower + Poison hath residence, and medicine power; + For this, being smelt, with that part cheers each part; + Being tasted, slays all senses with the heart. + Two such opposed kings encamp them still + In man as well as herbs- grace and rude will; + And where the worser is predominant, + Full soon the canker death eats up that plant. + + Enter Romeo. + + + Rom. Good morrow, father. + + Friar. Benedicite! + What early tongue so sweet saluteth me? + Young son, it argues a distempered head + So soon to bid good morrow to thy bed. + Care keeps his watch in every old man's eye, + And where care lodges sleep will never lie; + But where unbruised youth with unstuff'd brain + Doth couch his limbs, there golden sleep doth reign. + Therefore thy earliness doth me assure + Thou art uprous'd with some distemp'rature; + Or if not so, then here I hit it right- + Our Romeo hath not been in bed to-night. + + Rom. That last is true-the sweeter rest was mine. + + Friar. God pardon sin! Wast thou with Rosaline? + + Rom. With Rosaline, my ghostly father? No. + I have forgot that name, and that name's woe. + + Friar. That's my good son! But where hast thou been then? + + Rom. I'll tell thee ere thou ask it me again. + I have been feasting with mine enemy, + Where on a sudden one hath wounded me + That's by me wounded. Both our remedies + Within thy help and holy physic lies. + I bear no hatred, blessed man, for, lo, + My intercession likewise steads my foe. + + Friar. Be plain, good son, and homely in thy drift + Riddling confession finds but riddling shrift. + + Rom. Then plainly know my heart's dear love is set + On the fair daughter of rich Capulet; + As mine on hers, so hers is set on mine, + And all combin'd, save what thou must combine + By holy marriage. When, and where, and how + We met, we woo'd, and made exchange of vow, + I'll tell thee as we pass; but this I pray, + That thou consent to marry us to-day. + + Friar. Holy Saint Francis! What a change is here! + Is Rosaline, that thou didst love so dear, + So soon forsaken? Young men's love then lies + Not truly in their hearts, but in their eyes. + Jesu Maria! What a deal of brine + Hath wash'd thy sallow cheeks for Rosaline! + How much salt water thrown away in waste, + To season love, that of it doth not taste! + The sun not yet thy sighs from heaven clears, + Thy old groans ring yet in mine ancient ears. + Lo, here upon thy cheek the stain doth sit + Of an old tear that is not wash'd off yet. + If e'er thou wast thyself, and these woes thine, + Thou and these woes were all for Rosaline. + And art thou chang'd? Pronounce this sentence then: + Women may fall when there's no strength in men. + + Rom. Thou chid'st me oft for loving Rosaline. + + Friar. For doting, not for loving, pupil mine. + + Rom. And bad'st me bury love. + + Friar. Not in a grave + To lay one in, another out to have. + + Rom. I pray thee chide not. She whom I love now + Doth grace for grace and love for love allow. + The other did not so. + + Friar. O, she knew well + Thy love did read by rote, that could not spell. + But come, young waverer, come go with me. + In one respect I'll thy assistant be; + For this alliance may so happy prove + To turn your households' rancour to pure love. + + Rom. O, let us hence! I stand on sudden haste. + + Friar. Wisely, and slow. They stumble that run fast. + Exeunt. + + + + +Scene IV. +A street. + +Enter Benvolio and Mercutio. + + + Mer. Where the devil should this Romeo be? + Came he not home to-night? + + Ben. Not to his father's. I spoke with his man. + + Mer. Why, that same pale hard-hearted wench, that Rosaline, + Torments him so that he will sure run mad. + + Ben. Tybalt, the kinsman to old Capulet, + Hath sent a letter to his father's house. + + Mer. A challenge, on my life. + + Ben. Romeo will answer it. + + Mer. Any man that can write may answer a letter. + + Ben. Nay, he will answer the letter's master, how he dares, + being dared. + + Mer. Alas, poor Romeo, he is already dead! stabb'd with a white + wench's black eye; shot through the ear with a love song; the + very pin of his heart cleft with the blind bow-boy's + butt-shaft; and is he a man to encounter Tybalt? + + Ben. Why, what is Tybalt? + + Mer. More than Prince of Cats, I can tell you. O, he's the + courageous captain of compliments. He fights as you sing + pricksong-keeps time, distance, and proportion; rests me his + minim rest, one, two, and the third in your bosom! the very + butcher of a silk button, a duellist, a duellist! a gentleman + of the very first house, of the first and second cause. Ah, the + immortal passado! the punto reverse! the hay. + + Ben. The what? + + Mer. The pox of such antic, lisping, affecting fantasticoes- + these new tuners of accent! 'By Jesu, a very good blade! a very + tall man! a very good whore!' Why, is not this a lamentable thing, + grandsir, that we should be thus afflicted with these strange + flies, these fashion-mongers, these pardona-mi's, who stand + so much on the new form that they cannot sit at ease on the old + bench? O, their bones, their bones! + + Enter Romeo. + + + Ben. Here comes Romeo! here comes Romeo! + + Mer. Without his roe, like a dried herring. O flesh, flesh, how + art thou fishified! Now is he for the numbers that Petrarch + flowed in. Laura, to his lady, was but a kitchen wench (marry, she + had a better love to berhyme her), Dido a dowdy, Cleopatra a gypsy, + Helen and Hero hildings and harlots, This be a gray eye or so, + but not to the purpose. Signior Romeo, bon jour! There's a French + salutation to your French slop. You gave us the counterfeit + fairly last night. + + Rom. Good morrow to you both. What counterfeit did I give you? + + Mer. The slip, sir, the slip. Can you not conceive? + + Rom. Pardon, good Mercutio. My business was great, and in such a + case as mine a man may strain courtesy. + + Mer. That's as much as to say, such a case as yours constrains a + man to bow in the hams. + + Rom. Meaning, to cursy. + + Mer. Thou hast most kindly hit it. + + Rom. A most courteous exposition. + + Mer. Nay, I am the very pink of courtesy. + + Rom. Pink for flower. + + Mer. Right. + + Rom. Why, then is my pump well-flower'd. + + Mer. Well said! Follow me this jest now till thou hast worn out + thy pump, that, when the single sole of it is worn, the jest may + remain, after the wearing, solely singular. + + Rom. O single-sold jest, solely singular for the singleness! + + Mer. Come between us, good Benvolio! My wits faint. + + Rom. Swits and spurs, swits and spurs! or I'll cry a match. + + Mer. Nay, if our wits run the wild-goose chase, I am done; for + thou hast more of the wild goose in one of thy wits than, I am + sure, I have in my whole five. Was I with you there for the goose? + + Rom. Thou wast never with me for anything when thou wast not + there for the goose. + + Mer. I will bite thee by the ear for that jest. + + Rom. Nay, good goose, bite not! + + Mer. Thy wit is a very bitter sweeting; it is a most sharp sauce. + + Rom. And is it not, then, well serv'd in to a sweet goose? + + Mer. O, here's a wit of cheveril, that stretches from an inch + narrow to an ell broad! + + Rom. I stretch it out for that word 'broad,' which, added to + the goose, proves thee far and wide a broad goose. + + Mer. Why, is not this better now than groaning for love? Now + art thou sociable, now art thou Romeo; now art thou what thou art, by + art as well as by nature. For this drivelling love is like a + great natural that runs lolling up and down to hide his bauble in + a hole. + + Ben. Stop there, stop there! + + Mer. Thou desirest me to stop in my tale against the hair. + + Ben. Thou wouldst else have made thy tale large. + + Mer. O, thou art deceiv'd! I would have made it short; for I + was come to the whole depth of my tale, and meant indeed to + occupy the argument no longer. + + Rom. Here's goodly gear! + + Enter Nurse and her Man [Peter]. + + + Mer. A sail, a sail! + + Ben. Two, two! a shirt and a smock. + + Nurse. Peter! + + Peter. Anon. + + Nurse. My fan, Peter. + + Mer. Good Peter, to hide her face; for her fan's the fairer face of + the two. + + Nurse. God ye good morrow, gentlemen. + + Mer. God ye good-den, fair gentlewoman. + + Nurse. Is it good-den? + + Mer. 'Tis no less, I tell ye; for the bawdy hand of the dial is + now upon the prick of noon. + + Nurse. Out upon you! What a man are you! + + Rom. One, gentlewoman, that God hath made for himself to mar. + + Nurse. By my troth, it is well said. 'For himself to mar,' + quoth 'a? Gentlemen, can any of you tell me where I may find the + young Romeo? + + Rom. I can tell you; but young Romeo will be older when you + have found him than he was when you sought him. I am the youngest + of that name, for fault of a worse. + + Nurse. You say well. + + Mer. Yea, is the worst well? Very well took, i' faith! wisely, + wisely. + + Nurse. If you be he, sir, I desire some confidence with you. + + Ben. She will endite him to some supper. + + Mer. A bawd, a bawd, a bawd! So ho! + + Rom. What hast thou found? + + Mer. No hare, sir; unless a hare, sir, in a lenten pie, that is + something stale and hoar ere it be spent + He walks by them and sings. + + An old hare hoar, + And an old hare hoar, + Is very good meat in Lent; + But a hare that is hoar + Is too much for a score + When it hoars ere it be spent. + + Romeo, will you come to your father's? We'll to dinner thither. + + Rom. I will follow you. + + Mer. Farewell, ancient lady. Farewell, + [sings] lady, lady, lady. + Exeunt Mercutio, Benvolio. + + Nurse. Marry, farewell! I Pray you, Sir, what saucy merchant + was this that was so full of his ropery? + + Rom. A gentleman, nurse, that loves to hear himself talk and + will speak more in a minute than he will stand to in a month. + + Nurse. An 'a speak anything against me, I'll take him down, an +'a + were lustier than he is, and twenty such jacks; and if I cannot, + I'll find those that shall. Scurvy knave! I am none of his + flirt-gills; I am none of his skains-mates. And thou must + stand by too, and suffer every knave to use me at his pleasure! + + Peter. I saw no man use you at his pleasure. If I had, my + weapon should quickly have been out, I warrant you. I dare draw as + soon as another man, if I see occasion in a good quarrel, and the + law on my side. + + Nurse. Now, afore God, I am so vexed that every part about me + quivers. Scurvy knave! Pray you, sir, a word; and, as I told you, + my young lady bid me enquire you out. What she bid me say, I + will keep to myself; but first let me tell ye, if ye should lead + her into a fool's paradise, as they say, it were a very gross kind of + behaviour, as they say; for the gentlewoman is young; and + therefore, if you should deal double with her, truly it were + an ill thing to be off'red to any gentlewoman, and very weak dealing. + + Rom. Nurse, commend me to thy lady and mistress. I protest unto + thee- + + Nurse. Good heart, and I faith I will tell her as much. Lord, + Lord! she will be a joyful woman. + + Rom. What wilt thou tell her, nurse? Thou dost not mark me. + + Nurse. I will tell her, sir, that you do protest, which, as I + take it, is a gentlemanlike offer. + + Rom. Bid her devise + Some means to come to shrift this afternoon; + And there she shall at Friar Laurence' cell + Be shriv'd and married. Here is for thy pains. + + Nurse. No, truly, sir; not a penny. + + Rom. Go to! I say you shall. + + Nurse. This afternoon, sir? Well, she shall be there. + + Rom. And stay, good nurse, behind the abbey wall. + Within this hour my man shall be with thee + And bring thee cords made like a tackled stair, + Which to the high topgallant of my joy + Must be my convoy in the secret night. + Farewell. Be trusty, and I'll quit thy pains. + Farewell. Commend me to thy mistress. + + Nurse. Now God in heaven bless thee! Hark you, sir. + + Rom. What say'st thou, my dear nurse? + + Nurse. Is your man secret? Did you ne'er hear say, + Two may keep counsel, putting one away? + + Rom. I warrant thee my man's as true as steel. + + Nurse. Well, sir, my mistress is the sweetest lady. Lord, Lord! + when 'twas a little prating thing- O, there is a nobleman in + town, one Paris, that would fain lay knife aboard; but she, + good soul, had as lieve see a toad, a very toad, as see him. I + anger her sometimes, and tell her that Paris is the properer man; + but I'll warrant you, when I say so, she looks as pale as any + clout in the versal world. Doth not rosemary and Romeo begin both + with a letter? + + Rom. Ay, nurse; what of that? Both with an R. + + Nurse. Ah, mocker! that's the dog's name. R is for the- No; I + know it begins with some other letter; and she hath the prettiest + sententious of it, of you and rosemary, that it would do you + good to hear it. + + Rom. Commend me to thy lady. + + Nurse. Ay, a thousand times. [Exit Romeo.] Peter! + + Peter. Anon. + + Nurse. Peter, take my fan, and go before, and apace. + Exeunt. + + + + +Scene V. +Capulet's orchard. + +Enter Juliet. + + + Jul. The clock struck nine when I did send the nurse; + In half an hour she 'promis'd to return. + Perchance she cannot meet him. That's not so. + O, she is lame! Love's heralds should be thoughts, + Which ten times faster glide than the sun's beams + Driving back shadows over low'ring hills. + Therefore do nimble-pinion'd doves draw Love, + And therefore hath the wind-swift Cupid wings. + Now is the sun upon the highmost hill + Of this day's journey, and from nine till twelve + Is three long hours; yet she is not come. + Had she affections and warm youthful blood, + She would be as swift in motion as a ball; + My words would bandy her to my sweet love, + And his to me, + But old folks, many feign as they were dead- + Unwieldy, slow, heavy and pale as lead. + + Enter Nurse [and Peter]. + + O God, she comes! O honey nurse, what news? + Hast thou met with him? Send thy man away. + + Nurse. Peter, stay at the gate. + [Exit Peter.] + + Jul. Now, good sweet nurse- O Lord, why look'st thou sad? + Though news be sad, yet tell them merrily; + If good, thou shamest the music of sweet news + By playing it to me with so sour a face. + + Nurse. I am aweary, give me leave awhile. + Fie, how my bones ache! What a jaunce have I had! + + Jul. I would thou hadst my bones, and I thy news. + Nay, come, I pray thee speak. Good, good nurse, speak. + + Nurse. Jesu, what haste! Can you not stay awhile? + Do you not see that I am out of breath? + + Jul. How art thou out of breath when thou hast breath + To say to me that thou art out of breath? + The excuse that thou dost make in this delay + Is longer than the tale thou dost excuse. + Is thy news good or bad? Answer to that. + Say either, and I'll stay the circumstance. + Let me be satisfied, is't good or bad? + + Nurse. Well, you have made a simple choice; you know not how to + choose a man. Romeo? No, not he. Though his face be better + than any man's, yet his leg excels all men's; and for a hand and a + foot, and a body, though they be not to be talk'd on, yet + they are past compare. He is not the flower of courtesy, but, I'll + warrant him, as gentle as a lamb. Go thy ways, wench; serve +God. + What, have you din'd at home? + + Jul. No, no. But all this did I know before. + What says he of our marriage? What of that? + + Nurse. Lord, how my head aches! What a head have I! + It beats as it would fall in twenty pieces. + My back o' t' other side,- ah, my back, my back! + Beshrew your heart for sending me about + To catch my death with jauncing up and down! + + Jul. I' faith, I am sorry that thou art not well. + Sweet, sweet, Sweet nurse, tell me, what says my love? + + Nurse. Your love says, like an honest gentleman, and a courteous, + and a kind, and a handsome; and, I warrant, a virtuous- Where + is your mother? + + Jul. Where is my mother? Why, she is within. + Where should she be? How oddly thou repliest! + 'Your love says, like an honest gentleman, + "Where is your mother?"' + + Nurse. O God's Lady dear! + Are you so hot? Marry come up, I trow. + Is this the poultice for my aching bones? + Henceforward do your messages yourself. + + Jul. Here's such a coil! Come, what says Romeo? + + Nurse. Have you got leave to go to shrift to-day? + + Jul. I have. + + Nurse. Then hie you hence to Friar Laurence' cell; + There stays a husband to make you a wife. + Now comes the wanton blood up in your cheeks: + They'll be in scarlet straight at any news. + Hie you to church; I must another way, + To fetch a ladder, by the which your love + Must climb a bird's nest soon when it is dark. + I am the drudge, and toil in your delight; + But you shall bear the burthen soon at night. + Go; I'll to dinner; hie you to the cell. + + Jul. Hie to high fortune! Honest nurse, farewell. + Exeunt. + + + + +Scene VI. +Friar Laurence's cell. + +Enter Friar [Laurence] and Romeo. + + + Friar. So smile the heavens upon this holy act + That after-hours with sorrow chide us not! + + Rom. Amen, amen! But come what sorrow can, + It cannot countervail the exchange of joy + That one short minute gives me in her sight. + Do thou but close our hands with holy words, + Then love-devouring death do what he dare- + It is enough I may but call her mine. + + Friar. These violent delights have violent ends + And in their triumph die, like fire and powder, + Which, as they kiss, consume. The sweetest honey + Is loathsome in his own deliciousness + And in the taste confounds the appetite. + Therefore love moderately: long love doth so; + Too swift arrives as tardy as too slow. + + Enter Juliet. + + Here comes the lady. O, so light a foot + Will ne'er wear out the everlasting flint. + A lover may bestride the gossamer + That idles in the wanton summer air, + And yet not fall; so light is vanity. + + Jul. Good even to my ghostly confessor. + + Friar. Romeo shall thank thee, daughter, for us both. + + Jul. As much to him, else is his thanks too much. + + Rom. Ah, Juliet, if the measure of thy joy + Be heap'd like mine, and that thy skill be more + To blazon it, then sweeten with thy breath + This neighbour air, and let rich music's tongue + Unfold the imagin'd happiness that both + Receive in either by this dear encounter. + + Jul. Conceit, more rich in matter than in words, + Brags of his substance, not of ornament. + They are but beggars that can count their worth; + But my true love is grown to such excess + cannot sum up sum of half my wealth. + + Friar. Come, come with me, and we will make short work; + For, by your leaves, you shall not stay alone + Till Holy Church incorporate two in one. + [Exeunt.] + + + + +ACT III. Scene I. +A public place. + +Enter Mercutio, Benvolio, and Men. + + + Ben. I pray thee, good Mercutio, let's retire. + The day is hot, the Capulets abroad. + And if we meet, we shall not scape a brawl, + For now, these hot days, is the mad blood stirring. + + Mer. Thou art like one of these fellows that, when he enters + the confines of a tavern, claps me his sword upon the table and + says 'God send me no need of thee!' and by the operation of the + second cup draws him on the drawer, when indeed there is no need. + + Ben. Am I like such a fellow? + + Mer. Come, come, thou art as hot a jack in thy mood as any in + Italy; and as soon moved to be moody, and as soon moody to be + moved. + + Ben. And what to? + + Mer. Nay, an there were two such, we should have none shortly, + for one would kill the other. Thou! why, thou wilt quarrel with a + man that hath a hair more or a hair less in his beard than thou hast. + Thou wilt quarrel with a man for cracking nuts, having no + other reason but because thou hast hazel eyes. What eye but such an + eye would spy out such a quarrel? Thy head is as full of quarrels + as an egg is full of meat; and yet thy head hath been beaten as + addle as an egg for quarrelling. Thou hast quarrell'd with a + man for coughing in the street, because he hath wakened thy dog + that hath lain asleep in the sun. Didst thou not fall out with a + tailor for wearing his new doublet before Easter, with + another for tying his new shoes with an old riband? And yet thou wilt + tutor me from quarrelling! + + Ben. An I were so apt to quarrel as thou art, any man should + buy the fee simple of my life for an hour and a quarter. + + Mer. The fee simple? O simple! + + Enter Tybalt and others. + + + Ben. By my head, here come the Capulets. + + Mer. By my heel, I care not. + + Tyb. Follow me close, for I will speak to them. + Gentlemen, good den. A word with one of you. + + Mer. And but one word with one of us? + Couple it with something; make it a word and a blow. + + Tyb. You shall find me apt enough to that, sir, an you will give me + occasion. + + Mer. Could you not take some occasion without giving + + Tyb. Mercutio, thou consortest with Romeo. + + Mer. Consort? What, dost thou make us minstrels? An thou make + minstrels of us, look to hear nothing but discords. Here's my + fiddlestick; here's that shall make you dance. Zounds, consort! + + Ben. We talk here in the public haunt of men. + Either withdraw unto some private place + And reason coldly of your grievances, + Or else depart. Here all eyes gaze on us. + + Mer. Men's eyes were made to look, and let them gaze. + I will not budge for no man's pleasure, + + Enter Romeo. + + + Tyb. Well, peace be with you, sir. Here comes my man. + + Mer. But I'll be hang'd, sir, if he wear your livery. + Marry, go before to field, he'll be your follower! + Your worship in that sense may call him man. + + Tyb. Romeo, the love I bear thee can afford + No better term than this: thou art a villain. + + Rom. Tybalt, the reason that I have to love thee + Doth much excuse the appertaining rage + To such a greeting. Villain am I none. + Therefore farewell. I see thou knowest me not. + + Tyb. Boy, this shall not excuse the injuries + That thou hast done me; therefore turn and draw. + + Rom. I do protest I never injur'd thee, + But love thee better than thou canst devise + Till thou shalt know the reason of my love; + And so good Capulet, which name I tender + As dearly as mine own, be satisfied. + + Mer. O calm, dishonourable, vile submission! + Alla stoccata carries it away. [Draws.] + Tybalt, you ratcatcher, will you walk? + + Tyb. What wouldst thou have with me? + + Mer. Good King of Cats, nothing but one of your nine lives. +That I + mean to make bold withal, and, as you shall use me hereafter, + + dry-beat the rest of the eight. Will you pluck your sword out + of his pitcher by the ears? Make haste, lest mine be about your + ears ere it be out. + + Tyb. I am for you. [Draws.] + + Rom. Gentle Mercutio, put thy rapier up. + + Mer. Come, sir, your passado! + [They fight.] + + Rom. Draw, Benvolio; beat down their weapons. + Gentlemen, for shame! forbear this outrage! + Tybalt, Mercutio, the Prince expressly hath + Forbid this bandying in Verona streets. + Hold, Tybalt! Good Mercutio! + Tybalt under Romeo's arm thrusts Mercutio in, and flies + [with his Followers]. + + Mer. I am hurt. + A plague o' both your houses! I am sped. + Is he gone and hath nothing? + + Ben. What, art thou hurt? + + Mer. Ay, ay, a scratch, a scratch. Marry, 'tis enough. + Where is my page? Go, villain, fetch a surgeon. + [Exit Page.] + + Rom. Courage, man. The hurt cannot be much. + + Mer. No, 'tis not so deep as a well, nor so wide as a church door; + but 'tis enough, 'twill serve. Ask for me to-morrow, and you + shall find me a grave man. I am peppered, I warrant, for this + world. A plague o' both your houses! Zounds, a dog, a rat, a + mouse, a cat, to scratch a man to death! a braggart, a rogue, +a + villain, that fights by the book of arithmetic! Why the devil + came you between us? I was hurt under your arm. + + Rom. I thought all for the best. + + Mer. Help me into some house, Benvolio, + Or I shall faint. A plague o' both your houses! + They have made worms' meat of me. I have it, + And soundly too. Your houses! + [Exit. [supported by Benvolio]. + + Rom. This gentleman, the Prince's near ally, + My very friend, hath got this mortal hurt + In my behalf- my reputation stain'd + With Tybalt's slander- Tybalt, that an hour + Hath been my kinsman. O sweet Juliet, + Thy beauty hath made me effeminate + And in my temper soft'ned valour's steel + + Enter Benvolio. + + + Ben. O Romeo, Romeo, brave Mercutio's dead! + That gallant spirit hath aspir'd the clouds, + Which too untimely here did scorn the earth. + + Rom. This day's black fate on moe days doth depend; + This but begins the woe others must end. + + Enter Tybalt. + + + Ben. Here comes the furious Tybalt back again. + + Rom. Alive in triumph, and Mercutio slain? + Away to heaven respective lenity, + And fire-ey'd fury be my conduct now! + Now, Tybalt, take the 'villain' back again + That late thou gavest me; for Mercutio's soul + Is but a little way above our heads, + Staying for thine to keep him company. + Either thou or I, or both, must go with him. + + Tyb. Thou, wretched boy, that didst consort him here, + Shalt with him hence. + + Rom. This shall determine that. + They fight. Tybalt falls. + + Ben. Romeo, away, be gone! + The citizens are up, and Tybalt slain. + Stand not amaz'd. The Prince will doom thee death + If thou art taken. Hence, be gone, away! + + Rom. O, I am fortune's fool! + + Ben. Why dost thou stay? + Exit Romeo. + Enter Citizens. + + + Citizen. Which way ran he that kill'd Mercutio? + Tybalt, that murtherer, which way ran he? + + Ben. There lies that Tybalt. + + Citizen. Up, sir, go with me. + I charge thee in the Prince's name obey. + + + Enter Prince [attended], Old Montague, Capulet, their Wives, + and [others]. + + + Prince. Where are the vile beginners of this fray? + + Ben. O noble Prince. I can discover all + The unlucky manage of this fatal brawl. + There lies the man, slain by young Romeo, + That slew thy kinsman, brave Mercutio. + + Cap. Wife. Tybalt, my cousin! O my brother's child! + O Prince! O husband! O, the blood is spill'd + Of my dear kinsman! Prince, as thou art true, + For blood of ours shed blood of Montague. + O cousin, cousin! + + Prince. Benvolio, who began this bloody fray? + + Ben. Tybalt, here slain, whom Romeo's hand did stay. + Romeo, that spoke him fair, bid him bethink + How nice the quarrel was, and urg'd withal + Your high displeasure. All this- uttered + With gentle breath, calm look, knees humbly bow'd- + Could not take truce with the unruly spleen + Of Tybalt deaf to peace, but that he tilts + With piercing steel at bold Mercutio's breast; + Who, all as hot, turns deadly point to point, + And, with a martial scorn, with one hand beats + Cold death aside and with the other sends + It back to Tybalt, whose dexterity + Retorts it. Romeo he cries aloud, + 'Hold, friends! friends, part!' and swifter than his tongue, + His agile arm beats down their fatal points, + And 'twixt them rushes; underneath whose arm + An envious thrust from Tybalt hit the life + Of stout Mercutio, and then Tybalt fled; + But by-and-by comes back to Romeo, + Who had but newly entertain'd revenge, + And to't they go like lightning; for, ere I + Could draw to part them, was stout Tybalt slain; + And, as he fell, did Romeo turn and fly. + This is the truth, or let Benvolio die. + + Cap. Wife. He is a kinsman to the Montague; + Affection makes him false, he speaks not true. + Some twenty of them fought in this black strife, + And all those twenty could but kill one life. + I beg for justice, which thou, Prince, must give. + Romeo slew Tybalt; Romeo must not live. + + Prince. Romeo slew him; he slew Mercutio. + Who now the price of his dear blood doth owe? + + Mon. Not Romeo, Prince; he was Mercutio's friend; + His fault concludes but what the law should end, + The life of Tybalt. + + Prince. And for that offence + Immediately we do exile him hence. + I have an interest in your hate's proceeding, + My blood for your rude brawls doth lie a-bleeding; + But I'll amerce you with so strong a fine + That you shall all repent the loss of mine. + I will be deaf to pleading and excuses; + Nor tears nor prayers shall purchase out abuses. + Therefore use none. Let Romeo hence in haste, + Else, when he is found, that hour is his last. + Bear hence this body, and attend our will. + Mercy but murders, pardoning those that kill. + Exeunt. + + + + +Scene II. +Capulet's orchard. + +Enter Juliet alone. + + + Jul. Gallop apace, you fiery-footed steeds, + Towards Phoebus' lodging! Such a wagoner + As Phaeton would whip you to the West + And bring in cloudy night immediately. + Spread thy close curtain, love-performing night, + That runaway eyes may wink, and Romeo + Leap to these arms untalk'd of and unseen. + Lovers can see to do their amorous rites + By their own beauties; or, if love be blind, + It best agrees with night. Come, civil night, + Thou sober-suited matron, all in black, + And learn me how to lose a winning match, + Play'd for a pair of stainless maidenhoods. + Hood my unmann'd blood, bating in my cheeks, + With thy black mantle till strange love, grown bold, + Think true love acted simple modesty. + Come, night; come, Romeo; come, thou day in night; + For thou wilt lie upon the wings of night + Whiter than new snow upon a raven's back. + Come, gentle night; come, loving, black-brow'd night; + Give me my Romeo; and, when he shall die, + Take him and cut him out in little stars, + And he will make the face of heaven so fine + That all the world will be in love with night + And pay no worship to the garish sun. + O, I have bought the mansion of a love, + But not possess'd it; and though I am sold, + Not yet enjoy'd. So tedious is this day + As is the night before some festival + To an impatient child that hath new robes + And may not wear them. O, here comes my nurse, + + Enter Nurse, with cords. + + And she brings news; and every tongue that speaks + But Romeo's name speaks heavenly eloquence. + Now, nurse, what news? What hast thou there? the cords + That Romeo bid thee fetch? + + Nurse. Ay, ay, the cords. + [Throws them down.] + + Jul. Ay me! what news? Why dost thou wring thy hands + + Nurse. Ah, weraday! he's dead, he's dead, he's dead! + We are undone, lady, we are undone! + Alack the day! he's gone, he's kill'd, he's dead! + + Jul. Can heaven be so envious? + + Nurse. Romeo can, + Though heaven cannot. O Romeo, Romeo! + Who ever would have thought it? Romeo! + + Jul. What devil art thou that dost torment me thus? + This torture should be roar'd in dismal hell. + Hath Romeo slain himself? Say thou but 'I,' + And that bare vowel 'I' shall poison more + Than the death-darting eye of cockatrice. + I am not I, if there be such an 'I'; + Or those eyes shut that make thee answer 'I.' + If be be slain, say 'I'; or if not, 'no.' + Brief sounds determine of my weal or woe. + + Nurse. I saw the wound, I saw it with mine eyes, + (God save the mark!) here on his manly breast. + A piteous corse, a bloody piteous corse; + Pale, pale as ashes, all bedaub'd in blood, + All in gore-blood. I swounded at the sight. + + Jul. O, break, my heart! poor bankrout, break at once! + To prison, eyes; ne'er look on liberty! + Vile earth, to earth resign; end motion here, + And thou and Romeo press one heavy bier! + + Nurse. O Tybalt, Tybalt, the best friend I had! + O courteous Tybalt! honest gentleman + That ever I should live to see thee dead! + + Jul. What storm is this that blows so contrary? + Is Romeo slaught'red, and is Tybalt dead? + My dear-lov'd cousin, and my dearer lord? + Then, dreadful trumpet, sound the general doom! + For who is living, if those two are gone? + + Nurse. Tybalt is gone, and Romeo banished; + Romeo that kill'd him, he is banished. + + Jul. O God! Did Romeo's hand shed Tybalt's blood? + + Nurse. It did, it did! alas the day, it did! + + Jul. O serpent heart, hid with a flow'ring face! + Did ever dragon keep so fair a cave? + Beautiful tyrant! fiend angelical! + Dove-feather'd raven! wolvish-ravening lamb! + Despised substance of divinest show! + Just opposite to what thou justly seem'st- + A damned saint, an honourable villain! + O nature, what hadst thou to do in hell + When thou didst bower the spirit of a fiend + In mortal paradise of such sweet flesh? + Was ever book containing such vile matter + So fairly bound? O, that deceit should dwell + In such a gorgeous palace! + + Nurse. There's no trust, + No faith, no honesty in men; all perjur'd, + All forsworn, all naught, all dissemblers. + Ah, where's my man? Give me some aqua vitae. + These griefs, these woes, these sorrows make me old. + Shame come to Romeo! + + Jul. Blister'd be thy tongue + For such a wish! He was not born to shame. + Upon his brow shame is asham'd to sit; + For 'tis a throne where honour may be crown'd + Sole monarch of the universal earth. + O, what a beast was I to chide at him! + + Nurse. Will you speak well of him that kill'd your cousin? + + Jul. Shall I speak ill of him that is my husband? + Ah, poor my lord, what tongue shall smooth thy name + When I, thy three-hours wife, have mangled it? + But wherefore, villain, didst thou kill my cousin? + That villain cousin would have kill'd my husband. + Back, foolish tears, back to your native spring! + Your tributary drops belong to woe, + Which you, mistaking, offer up to joy. + My husband lives, that Tybalt would have slain; + And Tybalt's dead, that would have slain my husband. + All this is comfort; wherefore weep I then? + Some word there was, worser than Tybalt's death, + That murd'red me. I would forget it fain; + But O, it presses to my memory + Like damned guilty deeds to sinners' minds! + 'Tybalt is dead, and Romeo- banished.' + That 'banished,' that one word 'banished,' + Hath slain ten thousand Tybalts. Tybalt's death + Was woe enough, if it had ended there; + Or, if sour woe delights in fellowship + And needly will be rank'd with other griefs, + Why followed not, when she said 'Tybalt's dead,' + Thy father, or thy mother, nay, or both, + Which modern lamentation might have mov'd? + But with a rearward following Tybalt's death, + 'Romeo is banished'- to speak that word + Is father, mother, Tybalt, Romeo, Juliet, + All slain, all dead. 'Romeo is banished'- + There is no end, no limit, measure, bound, + In that word's death; no words can that woe sound. + Where is my father and my mother, nurse? + + Nurse. Weeping and wailing over Tybalt's corse. + Will you go to them? I will bring you thither. + + Jul. Wash they his wounds with tears? Mine shall be spent, + When theirs are dry, for Romeo's banishment. + Take up those cords. Poor ropes, you are beguil'd, + Both you and I, for Romeo is exil'd. + He made you for a highway to my bed; + But I, a maid, die maiden-widowed. + Come, cords; come, nurse. I'll to my wedding bed; + And death, not Romeo, take my maidenhead! + + Nurse. Hie to your chamber. I'll find Romeo + To comfort you. I wot well where he is. + Hark ye, your Romeo will be here at night. + I'll to him; he is hid at Laurence' cell. + + Jul. O, find him! give this ring to my true knight + And bid him come to take his last farewell. + Exeunt. + + + + +Scene III. +Friar Laurence's cell. + +Enter Friar [Laurence]. + + + Friar. Romeo, come forth; come forth, thou fearful man. + Affliction is enanmour'd of thy parts, + And thou art wedded to calamity. + + Enter Romeo. + + + Rom. Father, what news? What is the Prince's doom + What sorrow craves acquaintance at my hand + That I yet know not? + + Friar. Too familiar + Is my dear son with such sour company. + I bring thee tidings of the Prince's doom. + + Rom. What less than doomsday is the Prince's doom? + + Friar. A gentler judgment vanish'd from his lips- + Not body's death, but body's banishment. + + Rom. Ha, banishment? Be merciful, say 'death'; + For exile hath more terror in his look, + Much more than death. Do not say 'banishment.' + + Friar. Hence from Verona art thou banished. + Be patient, for the world is broad and wide. + + Rom. There is no world without Verona walls, + But purgatory, torture, hell itself. + Hence banished is banish'd from the world, + And world's exile is death. Then 'banishment' + Is death misterm'd. Calling death 'banishment,' + Thou cut'st my head off with a golden axe + And smilest upon the stroke that murders me. + + Friar. O deadly sin! O rude unthankfulness! + Thy fault our law calls death; but the kind Prince, + Taking thy part, hath rush'd aside the law, + And turn'd that black word death to banishment. + This is dear mercy, and thou seest it not. + + Rom. 'Tis torture, and not mercy. Heaven is here, + Where Juliet lives; and every cat and dog + And little mouse, every unworthy thing, + Live here in heaven and may look on her; + But Romeo may not. More validity, + More honourable state, more courtship lives + In carrion flies than Romeo. They may seize + On the white wonder of dear Juliet's hand + And steal immortal blessing from her lips, + Who, even in pure and vestal modesty, + Still blush, as thinking their own kisses sin; + But Romeo may not- he is banished. + This may flies do, when I from this must fly; + They are free men, but I am banished. + And sayest thou yet that exile is not death? + Hadst thou no poison mix'd, no sharp-ground knife, + No sudden mean of death, though ne'er so mean, + But 'banished' to kill me- 'banished'? + O friar, the damned use that word in hell; + Howling attends it! How hast thou the heart, + Being a divine, a ghostly confessor, + A sin-absolver, and my friend profess'd, + To mangle me with that word 'banished'? + + Friar. Thou fond mad man, hear me a little speak. + + Rom. O, thou wilt speak again of banishment. + + Friar. I'll give thee armour to keep off that word; + Adversity's sweet milk, philosophy, + To comfort thee, though thou art banished. + + Rom. Yet 'banished'? Hang up philosophy! + Unless philosophy can make a Juliet, + Displant a town, reverse a prince's doom, + It helps not, it prevails not. Talk no more. + + Friar. O, then I see that madmen have no ears. + + Rom. How should they, when that wise men have no eyes? + + Friar. Let me dispute with thee of thy estate. + + Rom. Thou canst not speak of that thou dost not feel. + Wert thou as young as I, Juliet thy love, + An hour but married, Tybalt murdered, + Doting like me, and like me banished, + Then mightst thou speak, then mightst thou tear thy hair, + And fall upon the ground, as I do now, + Taking the measure of an unmade grave. + Knock [within]. + + Friar. Arise; one knocks. Good Romeo, hide thyself. + + Rom. Not I; unless the breath of heartsick groans, + Mist-like infold me from the search of eyes. Knock. + + Friar. Hark, how they knock! Who's there? Romeo, arise; + Thou wilt be taken.- Stay awhile!- Stand up; Knock. + Run to my study.- By-and-by!- God's will, + What simpleness is this.- I come, I come! Knock. + Who knocks so hard? Whence come you? What's your will + + Nurse. [within] Let me come in, and you shall know my errand. + I come from Lady Juliet. + + Friar. Welcome then. + + Enter Nurse. + + + Nurse. O holy friar, O, tell me, holy friar + Where is my lady's lord, where's Romeo? + + Friar. There on the ground, with his own tears made drunk. + + Nurse. O, he is even in my mistress' case, + Just in her case! + + Friar. O woeful sympathy! + Piteous predicament! + + Nurse. Even so lies she, + Blubb'ring and weeping, weeping and blubbering. + Stand up, stand up! Stand, an you be a man. + For Juliet's sake, for her sake, rise and stand! + Why should you fall into so deep an O? + + Rom. (rises) Nurse- + + Nurse. Ah sir! ah sir! Well, death's the end of all. + + Rom. Spakest thou of Juliet? How is it with her? + Doth not she think me an old murtherer, + Now I have stain'd the childhood of our joy + With blood remov'd but little from her own? + Where is she? and how doth she! and what says + My conceal'd lady to our cancell'd love? + + Nurse. O, she says nothing, sir, but weeps and weeps; + And now falls on her bed, and then starts up, + And Tybalt calls; and then on Romeo cries, + And then down falls again. + + Rom. As if that name, + Shot from the deadly level of a gun, + Did murther her; as that name's cursed hand + Murder'd her kinsman. O, tell me, friar, tell me, + In what vile part of this anatomy + Doth my name lodge? Tell me, that I may sack + The hateful mansion. [Draws his dagger.] + + Friar. Hold thy desperate hand. + Art thou a man? Thy form cries out thou art; + Thy tears are womanish, thy wild acts denote + The unreasonable fury of a beast. + Unseemly woman in a seeming man! + Or ill-beseeming beast in seeming both! + Thou hast amaz'd me. By my holy order, + I thought thy disposition better temper'd. + Hast thou slain Tybalt? Wilt thou slay thyself? + And slay thy lady that in thy life lives, + By doing damned hate upon thyself? + Why railest thou on thy birth, the heaven, and earth? + Since birth and heaven and earth, all three do meet + In thee at once; which thou at once wouldst lose. + Fie, fie, thou shamest thy shape, thy love, thy wit, + Which, like a usurer, abound'st in all, + And usest none in that true use indeed + Which should bedeck thy shape, thy love, thy wit. + Thy noble shape is but a form of wax + Digressing from the valour of a man; + Thy dear love sworn but hollow perjury, + Killing that love which thou hast vow'd to cherish; + Thy wit, that ornament to shape and love, + Misshapen in the conduct of them both, + Like powder in a skilless soldier's flask, + is get afire by thine own ignorance, + And thou dismemb'red with thine own defence. + What, rouse thee, man! Thy Juliet is alive, + For whose dear sake thou wast but lately dead. + There art thou happy. Tybalt would kill thee, + But thou slewest Tybalt. There art thou happy too. + The law, that threat'ned death, becomes thy friend + And turns it to exile. There art thou happy. + A pack of blessings light upon thy back; + Happiness courts thee in her best array; + But, like a misbhav'd and sullen wench, + Thou pout'st upon thy fortune and thy love. + Take heed, take heed, for such die miserable. + Go get thee to thy love, as was decreed, + Ascend her chamber, hence and comfort her. + But look thou stay not till the watch be set, + For then thou canst not pass to Mantua, + Where thou shalt live till we can find a time + To blaze your marriage, reconcile your friends, + Beg pardon of the Prince, and call thee back + With twenty hundred thousand times more joy + Than thou went'st forth in lamentation. + Go before, nurse. Commend me to thy lady, + And bid her hasten all the house to bed, + Which heavy sorrow makes them apt unto. + Romeo is coming. + + Nurse. O Lord, I could have stay'd here all the night + To hear good counsel. O, what learning is! + My lord, I'll tell my lady you will come. + + Rom. Do so, and bid my sweet prepare to chide. + + Nurse. Here is a ring she bid me give you, sir. + Hie you, make haste, for it grows very late. Exit. + + Rom. How well my comfort is reviv'd by this! + + Friar. Go hence; good night; and here stands all your state: + Either be gone before the watch be set, + Or by the break of day disguis'd from hence. + Sojourn in Mantua. I'll find out your man, + And he shall signify from time to time + Every good hap to you that chances here. + Give me thy hand. 'Tis late. Farewell; good night. + + Rom. But that a joy past joy calls out on me, + It were a grief so brief to part with thee. + Farewell. + Exeunt. + + + + +Scene IV. +Capulet's house + +Enter Old Capulet, his Wife, and Paris. + + + Cap. Things have fall'n out, sir, so unluckily + That we have had no time to move our daughter. + Look you, she lov'd her kinsman Tybalt dearly, + And so did I. Well, we were born to die. + 'Tis very late; she'll not come down to-night. + I promise you, but for your company, + I would have been abed an hour ago. + + Par. These times of woe afford no tune to woo. + Madam, good night. Commend me to your daughter. + + Lady. I will, and know her mind early to-morrow; + To-night she's mew'd up to her heaviness. + + Cap. Sir Paris, I will make a desperate tender + Of my child's love. I think she will be rul'd + In all respects by me; nay more, I doubt it not. + Wife, go you to her ere you go to bed; + Acquaint her here of my son Paris' love + And bid her (mark you me?) on Wednesday next- + But, soft! what day is this? + + Par. Monday, my lord. + + Cap. Monday! ha, ha! Well, Wednesday is too soon. + Thursday let it be- a Thursday, tell her + She shall be married to this noble earl. + Will you be ready? Do you like this haste? + We'll keep no great ado- a friend or two; + For hark you, Tybalt being slain so late, + It may be thought we held him carelessly, + Being our kinsman, if we revel much. + Therefore we'll have some half a dozen friends, + And there an end. But what say you to Thursday? + + Par. My lord, I would that Thursday were to-morrow. + + Cap. Well, get you gone. A Thursday be it then. + Go you to Juliet ere you go to bed; + Prepare her, wife, against this wedding day. + Farewell, My lord.- Light to my chamber, ho! + Afore me, It is so very very late + That we may call it early by-and-by. + Good night. + Exeunt + + + + +Scene V. +Capulet's orchard. + +Enter Romeo and Juliet aloft, at the Window. + + + Jul. Wilt thou be gone? It is not yet near day. + It was the nightingale, and not the lark, + That pierc'd the fearful hollow of thine ear. + Nightly she sings on yond pomegranate tree. + Believe me, love, it was the nightingale. + + Rom. It was the lark, the herald of the morn; + No nightingale. Look, love, what envious streaks + Do lace the severing clouds in yonder East. + Night's candles are burnt out, and jocund day + Stands tiptoe on the misty mountain tops. + I must be gone and live, or stay and die. + + Jul. Yond light is not daylight; I know it, I. + It is some meteor that the sun exhales + To be to thee this night a torchbearer + And light thee on the way to Mantua. + Therefore stay yet; thou need'st not to be gone. + + Rom. Let me be ta'en, let me be put to death. + I am content, so thou wilt have it so. + I'll say yon grey is not the morning's eye, + 'Tis but the pale reflex of Cynthia's brow; + Nor that is not the lark whose notes do beat + The vaulty heaven so high above our heads. + I have more care to stay than will to go. + Come, death, and welcome! Juliet wills it so. + How is't, my soul? Let's talk; it is not day. + + Jul. It is, it is! Hie hence, be gone, away! + It is the lark that sings so out of tune, + Straining harsh discords and unpleasing sharps. + Some say the lark makes sweet division; + This doth not so, for she divideth us. + Some say the lark and loathed toad chang'd eyes; + O, now I would they had chang'd voices too, + Since arm from arm that voice doth us affray, + Hunting thee hence with hunt's-up to the day! + O, now be gone! More light and light it grows. + + Rom. More light and light- more dark and dark our woes! + + Enter Nurse. + + + Nurse. Madam! + + Jul. Nurse? + + Nurse. Your lady mother is coming to your chamber. + The day is broke; be wary, look about. + + Jul. Then, window, let day in, and let life out. + [Exit.] + + Rom. Farewell, farewell! One kiss, and I'll descend. + He goeth down. + + Jul. Art thou gone so, my lord, my love, my friend? + I must hear from thee every day in the hour, + For in a minute there are many days. + O, by this count I shall be much in years + Ere I again behold my Romeo! + + Rom. Farewell! + I will omit no opportunity + That may convey my greetings, love, to thee. + + Jul. O, think'st thou we shall ever meet again? + + Rom. I doubt it not; and all these woes shall serve + For sweet discourses in our time to come. + + Jul. O God, I have an ill-divining soul! + Methinks I see thee, now thou art below, + As one dead in the bottom of a tomb. + Either my eyesight fails, or thou look'st pale. + + Rom. And trust me, love, in my eye so do you. + Dry sorrow drinks our blood. Adieu, adieu! +Exit. + + Jul. O Fortune, Fortune! all men call thee fickle. + If thou art fickle, what dost thou with him + That is renown'd for faith? Be fickle, Fortune, + For then I hope thou wilt not keep him long + But send him back. + + Lady. [within] Ho, daughter! are you up? + + Jul. Who is't that calls? It is my lady mother. + Is she not down so late, or up so early? + What unaccustom'd cause procures her hither? + + Enter Mother. + + + Lady. Why, how now, Juliet? + + Jul. Madam, I am not well. + + Lady. Evermore weeping for your cousin's death? + What, wilt thou wash him from his grave with tears? + An if thou couldst, thou couldst not make him live. + Therefore have done. Some grief shows much of love; + But much of grief shows still some want of wit. + + Jul. Yet let me weep for such a feeling loss. + + Lady. So shall you feel the loss, but not the friend + Which you weep for. + + Jul. Feeling so the loss, + I cannot choose but ever weep the friend. + + Lady. Well, girl, thou weep'st not so much for his death + As that the villain lives which slaughter'd him. + + Jul. What villain, madam? + + Lady. That same villain Romeo. + + Jul. [aside] Villain and he be many miles asunder.- + God pardon him! I do, with all my heart; + And yet no man like he doth grieve my heart. + + Lady. That is because the traitor murderer lives. + + Jul. Ay, madam, from the reach of these my hands. + Would none but I might venge my cousin's death! + + Lady. We will have vengeance for it, fear thou not. + Then weep no more. I'll send to one in Mantua, + Where that same banish'd runagate doth live, + Shall give him such an unaccustom'd dram + That he shall soon keep Tybalt company; + And then I hope thou wilt be satisfied. + + Jul. Indeed I never shall be satisfied + With Romeo till I behold him- dead- + Is my poor heart so for a kinsman vex'd. + Madam, if you could find out but a man + To bear a poison, I would temper it; + That Romeo should, upon receipt thereof, + Soon sleep in quiet. O, how my heart abhors + To hear him nam'd and cannot come to him, + To wreak the love I bore my cousin Tybalt + Upon his body that hath slaughter'd him! + + Lady. Find thou the means, and I'll find such a man. + But now I'll tell thee joyful tidings, girl. + + Jul. And joy comes well in such a needy time. + What are they, I beseech your ladyship? + + Lady. Well, well, thou hast a careful father, child; + One who, to put thee from thy heaviness, + Hath sorted out a sudden day of joy + That thou expects not nor I look'd not for. + + Jul. Madam, in happy time! What day is that? + + Lady. Marry, my child, early next Thursday morn + The gallant, young, and noble gentleman, + The County Paris, at Saint Peter's Church, + Shall happily make thee there a joyful bride. + + Jul. Now by Saint Peter's Church, and Peter too, + He shall not make me there a joyful bride! + I wonder at this haste, that I must wed + Ere he that should be husband comes to woo. + I pray you tell my lord and father, madam, + I will not marry yet; and when I do, I swear + It shall be Romeo, whom you know I hate, + Rather than Paris. These are news indeed! + + Lady. Here comes your father. Tell him so yourself, + And see how be will take it at your hands. + + Enter Capulet and Nurse. + + + Cap. When the sun sets the air doth drizzle dew, + But for the sunset of my brother's son + It rains downright. + How now? a conduit, girl? What, still in tears? + Evermore show'ring? In one little body + Thou counterfeit'st a bark, a sea, a wind: + For still thy eyes, which I may call the sea, + Do ebb and flow with tears; the bark thy body is + Sailing in this salt flood; the winds, thy sighs, + Who, raging with thy tears and they with them, + Without a sudden calm will overset + Thy tempest-tossed body. How now, wife? + Have you delivered to her our decree? + + Lady. Ay, sir; but she will none, she gives you thanks. + I would the fool were married to her grave! + + Cap. Soft! take me with you, take me with you, wife. + How? Will she none? Doth she not give us thanks? + Is she not proud? Doth she not count her blest, + Unworthy as she is, that we have wrought + So worthy a gentleman to be her bridegroom? + + Jul. Not proud you have, but thankful that you have. + Proud can I never be of what I hate, + But thankful even for hate that is meant love. + + Cap. How, how, how, how, choplogic? What is this? + 'Proud'- and 'I thank you'- and 'I thank you not'- + And yet 'not proud'? Mistress minion you, + Thank me no thankings, nor proud me no prouds, + But fettle your fine joints 'gainst Thursday next + To go with Paris to Saint Peter's Church, + Or I will drag thee on a hurdle thither. + Out, you green-sickness carrion I out, you baggage! + You tallow-face! + + Lady. Fie, fie! what, are you mad? + + Jul. Good father, I beseech you on my knees, + Hear me with patience but to speak a word. + + Cap. Hang thee, young baggage! disobedient wretch! + I tell thee what- get thee to church a Thursday + Or never after look me in the face. + Speak not, reply not, do not answer me! + My fingers itch. Wife, we scarce thought us blest + That God had lent us but this only child; + But now I see this one is one too much, + And that we have a curse in having her. + Out on her, hilding! + + Nurse. God in heaven bless her! + You are to blame, my lord, to rate her so. + + Cap. And why, my Lady Wisdom? Hold your tongue, + Good Prudence. Smatter with your gossips, go! + + Nurse. I speak no treason. + + Cap. O, God-i-god-en! + + Nurse. May not one speak? + + Cap. Peace, you mumbling fool! + Utter your gravity o'er a gossip's bowl, + For here we need it not. + + Lady. You are too hot. + + Cap. God's bread I it makes me mad. Day, night, late, early, + At home, abroad, alone, in company, + Waking or sleeping, still my care hath been + To have her match'd; and having now provided + A gentleman of princely parentage, + Of fair demesnes, youthful, and nobly train'd, + Stuff'd, as they say, with honourable parts, + Proportion'd as one's thought would wish a man- + And then to have a wretched puling fool, + A whining mammet, in her fortune's tender, + To answer 'I'll not wed, I cannot love; + I am too young, I pray you pardon me'! + But, an you will not wed, I'll pardon you. + Graze where you will, you shall not house with me. + Look to't, think on't; I do not use to jest. + Thursday is near; lay hand on heart, advise: + An you be mine, I'll give you to my friend; + An you be not, hang, beg, starve, die in the streets, + For, by my soul, I'll ne'er acknowledge thee, + Nor what is mine shall never do thee good. + Trust to't. Bethink you. I'll not be forsworn. Exit. + + Jul. Is there no pity sitting in the clouds + That sees into the bottom of my grief? + O sweet my mother, cast me not away! + Delay this marriage for a month, a week; + Or if you do not, make the bridal bed + In that dim monument where Tybalt lies. + + Lady. Talk not to me, for I'll not speak a word. + Do as thou wilt, for I have done with thee. Exit. + + Jul. O God!- O nurse, how shall this be prevented? + My husband is on earth, my faith in heaven. + How shall that faith return again to earth + Unless that husband send it me from heaven + By leaving earth? Comfort me, counsel me. + Alack, alack, that heaven should practise stratagems + Upon so soft a subject as myself! + What say'st thou? Hast thou not a word of joy? + Some comfort, nurse. + + Nurse. Faith, here it is. + Romeo is banish'd; and all the world to nothing + That he dares ne'er come back to challenge you; + Or if he do, it needs must be by stealth. + Then, since the case so stands as now it doth, + I think it best you married with the County. + O, he's a lovely gentleman! + Romeo's a dishclout to him. An eagle, madam, + Hath not so green, so quick, so fair an eye + As Paris hath. Beshrew my very heart, + I think you are happy in this second match, + For it excels your first; or if it did not, + Your first is dead- or 'twere as good he were + As living here and you no use of him. + + Jul. Speak'st thou this from thy heart? + + Nurse. And from my soul too; else beshrew them both. + + Jul. Amen! + + Nurse. What? + + Jul. Well, thou hast comforted me marvellous much. + Go in; and tell my lady I am gone, + Having displeas'd my father, to Laurence' cell, + To make confession and to be absolv'd. + + Nurse. Marry, I will; and this is wisely done. Exit. + + Jul. Ancient damnation! O most wicked fiend! + Is it more sin to wish me thus forsworn, + Or to dispraise my lord with that same tongue + Which she hath prais'd him with above compare + So many thousand times? Go, counsellor! + Thou and my bosom henceforth shall be twain. + I'll to the friar to know his remedy. + If all else fail, myself have power to die. Exit. + + + + +ACT IV. Scene I. +Friar Laurence's cell. + +Enter Friar, [Laurence] and County Paris. + + + Friar. On Thursday, sir? The time is very short. + + Par. My father Capulet will have it so, + And I am nothing slow to slack his haste. + + Friar. You say you do not know the lady's mind. + Uneven is the course; I like it not. + + Par. Immoderately she weeps for Tybalt's death, + And therefore have I little talk'd of love; + For Venus smiles not in a house of tears. + Now, sir, her father counts it dangerous + That she do give her sorrow so much sway, + And in his wisdom hastes our marriage + To stop the inundation of her tears, + Which, too much minded by herself alone, + May be put from her by society. + Now do you know the reason of this haste. + + Friar. [aside] I would I knew not why it should be slow'd.- + Look, sir, here comes the lady toward my cell. + + Enter Juliet. + + + Par. Happily met, my lady and my wife! + + Jul. That may be, sir, when I may be a wife. + + Par. That may be must be, love, on Thursday next. + + Jul. What must be shall be. + + Friar. That's a certain text. + + Par. Come you to make confession to this father? + + Jul. To answer that, I should confess to you. + + Par. Do not deny to him that you love me. + + Jul. I will confess to you that I love him. + + Par. So will ye, I am sure, that you love me. + + Jul. If I do so, it will be of more price, + Being spoke behind your back, than to your face. + + Par. Poor soul, thy face is much abus'd with tears. + + Jul. The tears have got small victory by that, + For it was bad enough before their spite. + + Par. Thou wrong'st it more than tears with that report. + + Jul. That is no slander, sir, which is a truth; + And what I spake, I spake it to my face. + + Par. Thy face is mine, and thou hast sland'red it. + + Jul. It may be so, for it is not mine own. + Are you at leisure, holy father, now, + Or shall I come to you at evening mass + + Friar. My leisure serves me, pensive daughter, now. + My lord, we must entreat the time alone. + + Par. God shield I should disturb devotion! + Juliet, on Thursday early will I rouse ye. + Till then, adieu, and keep this holy kiss. Exit. + + Jul. O, shut the door! and when thou hast done so, + Come weep with me- past hope, past cure, past help! + + Friar. Ah, Juliet, I already know thy grief; + It strains me past the compass of my wits. + I hear thou must, and nothing may prorogue it, + On Thursday next be married to this County. + + Jul. Tell me not, friar, that thou hear'st of this, + Unless thou tell me how I may prevent it. + If in thy wisdom thou canst give no help, + Do thou but call my resolution wise + And with this knife I'll help it presently. + God join'd my heart and Romeo's, thou our hands; + And ere this hand, by thee to Romeo's seal'd, + Shall be the label to another deed, + Or my true heart with treacherous revolt + Turn to another, this shall slay them both. + Therefore, out of thy long-experienc'd time, + Give me some present counsel; or, behold, + 'Twixt my extremes and me this bloody knife + Shall play the empire, arbitrating that + Which the commission of thy years and art + Could to no issue of true honour bring. + Be not so long to speak. I long to die + If what thou speak'st speak not of remedy. + + Friar. Hold, daughter. I do spy a kind of hope, + Which craves as desperate an execution + As that is desperate which we would prevent. + If, rather than to marry County Paris + Thou hast the strength of will to slay thyself, + Then is it likely thou wilt undertake + A thing like death to chide away this shame, + That cop'st with death himself to scape from it; + And, if thou dar'st, I'll give thee remedy. + + Jul. O, bid me leap, rather than marry Paris, + From off the battlements of yonder tower, + Or walk in thievish ways, or bid me lurk + Where serpents are; chain me with roaring bears, + Or shut me nightly in a charnel house, + O'ercover'd quite with dead men's rattling bones, + With reeky shanks and yellow chapless skulls; + Or bid me go into a new-made grave + And hide me with a dead man in his shroud- + Things that, to hear them told, have made me tremble- + And I will do it without fear or doubt, + To live an unstain'd wife to my sweet love. + + Friar. Hold, then. Go home, be merry, give consent + To marry Paris. Wednesday is to-morrow. + To-morrow night look that thou lie alone; + Let not the nurse lie with thee in thy chamber. + Take thou this vial, being then in bed, + And this distilled liquor drink thou off; + When presently through all thy veins shall run + A cold and drowsy humour; for no pulse + Shall keep his native progress, but surcease; + No warmth, no breath, shall testify thou livest; + The roses in thy lips and cheeks shall fade + To paly ashes, thy eyes' windows fall + Like death when he shuts up the day of life; + Each part, depriv'd of supple government, + Shall, stiff and stark and cold, appear like death; + And in this borrowed likeness of shrunk death + Thou shalt continue two-and-forty hours, + And then awake as from a pleasant sleep. + Now, when the bridegroom in the morning comes + To rouse thee from thy bed, there art thou dead. + Then, as the manner of our country is, + In thy best robes uncovered on the bier + Thou shalt be borne to that same ancient vault + Where all the kindred of the Capulets lie. + In the mean time, against thou shalt awake, + Shall Romeo by my letters know our drift; + And hither shall he come; and he and I + Will watch thy waking, and that very night + Shall Romeo bear thee hence to Mantua. + And this shall free thee from this present shame, + If no inconstant toy nor womanish fear + Abate thy valour in the acting it. + + Jul. Give me, give me! O, tell not me of fear! + + Friar. Hold! Get you gone, be strong and prosperous + In this resolve. I'll send a friar with speed + To Mantua, with my letters to thy lord. + + Jul. Love give me strength! and strength shall help afford. + Farewell, dear father. + Exeunt. + + + + +Scene II. +Capulet's house. + +Enter Father Capulet, Mother, Nurse, and Servingmen, + two or three. + + + Cap. So many guests invite as here are writ. + [Exit a Servingman.] + Sirrah, go hire me twenty cunning cooks. + + Serv. You shall have none ill, sir; for I'll try if they can + lick their fingers. + + Cap. How canst thou try them so? + + Serv. Marry, sir, 'tis an ill cook that cannot lick his own + fingers. Therefore he that cannot lick his fingers goes not + with me. + + Cap. Go, begone. + Exit Servingman. + We shall be much unfurnish'd for this time. + What, is my daughter gone to Friar Laurence? + + Nurse. Ay, forsooth. + + Cap. Well, be may chance to do some good on her. + A peevish self-will'd harlotry it is. + + Enter Juliet. + + + Nurse. See where she comes from shrift with merry look. + + Cap. How now, my headstrong? Where have you been gadding? + + Jul. Where I have learnt me to repent the sin + Of disobedient opposition + To you and your behests, and am enjoin'd + By holy Laurence to fall prostrate here + To beg your pardon. Pardon, I beseech you! + Henceforward I am ever rul'd by you. + + Cap. Send for the County. Go tell him of this. + I'll have this knot knit up to-morrow morning. + + Jul. I met the youthful lord at Laurence' cell + And gave him what becomed love I might, + Not stepping o'er the bounds of modesty. + + Cap. Why, I am glad on't. This is well. Stand up. + This is as't should be. Let me see the County. + Ay, marry, go, I say, and fetch him hither. + Now, afore God, this reverend holy friar, + All our whole city is much bound to him. + + Jul. Nurse, will you go with me into my closet + To help me sort such needful ornaments + As you think fit to furnish me to-morrow? + + Mother. No, not till Thursday. There is time enough. + + Cap. Go, nurse, go with her. We'll to church to-morrow. + Exeunt Juliet and Nurse. + + Mother. We shall be short in our provision. + 'Tis now near night. + + Cap. Tush, I will stir about, + And all things shall be well, I warrant thee, wife. + Go thou to Juliet, help to deck up her. + I'll not to bed to-night; let me alone. + I'll play the housewife for this once. What, ho! + They are all forth; well, I will walk myself + To County Paris, to prepare him up + Against to-morrow. My heart is wondrous light, + Since this same wayward girl is so reclaim'd. + Exeunt. + + + + +Scene III. +Juliet's chamber. + +Enter Juliet and Nurse. + + + Jul. Ay, those attires are best; but, gentle nurse, + I pray thee leave me to myself to-night; + For I have need of many orisons + To move the heavens to smile upon my state, + Which, well thou knowest, is cross and full of sin. + + Enter Mother. + + + Mother. What, are you busy, ho? Need you my help? + + Jul. No, madam; we have cull'd such necessaries + As are behooffull for our state to-morrow. + So please you, let me now be left alone, + And let the nurse this night sit up with you; + For I am sure you have your hands full all + In this so sudden business. + + Mother. Good night. + Get thee to bed, and rest; for thou hast need. + Exeunt [Mother and Nurse.] + + Jul. Farewell! God knows when we shall meet again. + I have a faint cold fear thrills through my veins + That almost freezes up the heat of life. + I'll call them back again to comfort me. + Nurse!- What should she do here? + My dismal scene I needs must act alone. + Come, vial. + What if this mixture do not work at all? + Shall I be married then to-morrow morning? + No, No! This shall forbid it. Lie thou there. + Lays down a dagger. + What if it be a poison which the friar + Subtilly hath minist'red to have me dead, + Lest in this marriage he should be dishonour'd + Because he married me before to Romeo? + I fear it is; and yet methinks it should not, + For he hath still been tried a holy man. + I will not entertain so bad a thought. + How if, when I am laid into the tomb, + I wake before the time that Romeo + Come to redeem me? There's a fearful point! + Shall I not then be stifled in the vault, + To whose foul mouth no healthsome air breathes in, + And there die strangled ere my Romeo comes? + Or, if I live, is it not very like + The horrible conceit of death and night, + Together with the terror of the place- + As in a vault, an ancient receptacle + Where for this many hundred years the bones + Of all my buried ancestors are pack'd; + Where bloody Tybalt, yet but green in earth, + Lies fest'ring in his shroud; where, as they say, + At some hours in the night spirits resort- + Alack, alack, is it not like that I, + So early waking- what with loathsome smells, + And shrieks like mandrakes torn out of the earth, + That living mortals, hearing them, run mad- + O, if I wake, shall I not be distraught, + Environed with all these hideous fears, + And madly play with my forefathers' joints, + And pluck the mangled Tybalt from his shroud., + And, in this rage, with some great kinsman's bone + As with a club dash out my desp'rate brains? + O, look! methinks I see my cousin's ghost + Seeking out Romeo, that did spit his body + Upon a rapier's point. Stay, Tybalt, stay! + Romeo, I come! this do I drink to thee. + + She [drinks and] falls upon her bed within the curtains. + + + + +Scene IV. +Capulet's house. + +Enter Lady of the House and Nurse. + + + Lady. Hold, take these keys and fetch more spices, nurse. + + Nurse. They call for dates and quinces in the pastry. + + Enter Old Capulet. + + + Cap. Come, stir, stir, stir! The second cock hath crow'd, + The curfew bell hath rung, 'tis three o'clock. + Look to the bak'd meats, good Angelica; + Spare not for cost. + + Nurse. Go, you cot-quean, go, + Get you to bed! Faith, you'll be sick to-morrow + For this night's watching. + + Cap. No, not a whit. What, I have watch'd ere now + All night for lesser cause, and ne'er been sick. + + Lady. Ay, you have been a mouse-hunt in your time; + But I will watch you from such watching now. + Exeunt Lady and Nurse. + + Cap. A jealous hood, a jealous hood! + + + Enter three or four [Fellows, with spits and logs and baskets. + + What is there? Now, fellow, + + Fellow. Things for the cook, sir; but I know not what. + + Cap. Make haste, make haste. [Exit Fellow.] Sirrah, fetch drier + logs. + Call Peter; he will show thee where they are. + + Fellow. I have a head, sir, that will find out logs + And never trouble Peter for the matter. + + Cap. Mass, and well said; a merry whoreson, ha! + Thou shalt be loggerhead. [Exit Fellow.] Good faith, 'tis day. + The County will be here with music straight, + For so he said he would. Play music. + I hear him near. + Nurse! Wife! What, ho! What, nurse, I say! + + Enter Nurse. + Go waken Juliet; go and trim her up. + I'll go and chat with Paris. Hie, make haste, + Make haste! The bridegroom he is come already: + Make haste, I say. + [Exeunt.] + + + + +Scene V. +Juliet's chamber. + +[Enter Nurse.] + + + Nurse. Mistress! what, mistress! Juliet! Fast, I warrant her, she. + Why, lamb! why, lady! Fie, you slug-abed! + Why, love, I say! madam! sweetheart! Why, bride! + What, not a word? You take your pennyworths now! + Sleep for a week; for the next night, I warrant, + The County Paris hath set up his rest + That you shall rest but little. God forgive me! + Marry, and amen. How sound is she asleep! + I needs must wake her. Madam, madam, madam! + Ay, let the County take you in your bed! + He'll fright you up, i' faith. Will it not be? + [Draws aside the curtains.] + What, dress'd, and in your clothes, and down again? + I must needs wake you. Lady! lady! lady! + Alas, alas! Help, help! My lady's dead! + O weraday that ever I was born! + Some aqua-vitae, ho! My lord! my lady! + + Enter Mother. + + + Mother. What noise is here? + + Nurse. O lamentable day! + + Mother. What is the matter? + + Nurse. Look, look! O heavy day! + + Mother. O me, O me! My child, my only life! + Revive, look up, or I will die with thee! + Help, help! Call help. + + Enter Father. + + + Father. For shame, bring Juliet forth; her lord is come. + + Nurse. She's dead, deceas'd; she's dead! Alack the day! + + Mother. Alack the day, she's dead, she's dead, she's dead! + + Cap. Ha! let me see her. Out alas! she's cold, + Her blood is settled, and her joints are stiff; + Life and these lips have long been separated. + Death lies on her like an untimely frost + Upon the sweetest flower of all the field. + + Nurse. O lamentable day! + + Mother. O woful time! + + Cap. Death, that hath ta'en her hence to make me wail, + Ties up my tongue and will not let me speak. + + + Enter Friar [Laurence] and the County [Paris], with Musicians. + + + Friar. Come, is the bride ready to go to church? + + Cap. Ready to go, but never to return. + O son, the night before thy wedding day + Hath Death lain with thy wife. See, there she lies, + Flower as she was, deflowered by him. + Death is my son-in-law, Death is my heir; + My daughter he hath wedded. I will die + And leave him all. Life, living, all is Death's. + + Par. Have I thought long to see this morning's face, + And doth it give me such a sight as this? + + Mother. Accurs'd, unhappy, wretched, hateful day! + Most miserable hour that e'er time saw + In lasting labour of his pilgrimage! + But one, poor one, one poor and loving child, + But one thing to rejoice and solace in, + And cruel Death hath catch'd it from my sight! + + Nurse. O woe? O woful, woful, woful day! + Most lamentable day, most woful day + That ever ever I did yet behold! + O day! O day! O day! O hateful day! + Never was seen so black a day as this. + O woful day! O woful day! + + Par. Beguil'd, divorced, wronged, spited, slain! + Most detestable Death, by thee beguil'd, + By cruel cruel thee quite overthrown! + O love! O life! not life, but love in death + + Cap. Despis'd, distressed, hated, martyr'd, kill'd! + Uncomfortable time, why cam'st thou now + To murther, murther our solemnity? + O child! O child! my soul, and not my child! + Dead art thou, dead! alack, my child is dead, + And with my child my joys are buried! + + Friar. Peace, ho, for shame! Confusion's cure lives not + In these confusions. Heaven and yourself + Had part in this fair maid! now heaven hath all, + And all the better is it for the maid. + Your part in her you could not keep from death, + But heaven keeps his part in eternal life. + The most you sought was her promotion, + For 'twas your heaven she should be advanc'd; + And weep ye now, seeing she is advanc'd + Above the clouds, as high as heaven itself? + O, in this love, you love your child so ill + That you run mad, seeing that she is well. + She's not well married that lives married long, + But she's best married that dies married young. + Dry up your tears and stick your rosemary + On this fair corse, and, as the custom is, + In all her best array bear her to church; + For though fond nature bids us all lament, + Yet nature's tears are reason's merriment. + + Cap. All things that we ordained festival + Turn from their office to black funeral- + Our instruments to melancholy bells, + Our wedding cheer to a sad burial feast; + Our solemn hymns to sullen dirges change; + Our bridal flowers serve for a buried corse; + And all things change them to the contrary. + + Friar. Sir, go you in; and, madam, go with him; + And go, Sir Paris. Every one prepare + To follow this fair corse unto her grave. + The heavens do low'r upon you for some ill; + Move them no more by crossing their high will. + Exeunt. Manent Musicians [and Nurse]. + 1. Mus. Faith, we may put up our pipes and be gone. + + Nurse. Honest good fellows, ah, put up, put up! + For well you know this is a pitiful case. [Exit.] + 1. Mus. Ay, by my troth, the case may be amended. + + Enter Peter. + + + Pet. Musicians, O, musicians, 'Heart's ease,' 'Heart's ease'! + O, an you will have me live, play 'Heart's ease.' + 1. Mus. Why 'Heart's ease'', + + Pet. O, musicians, because my heart itself plays 'My heart is + full of woe.' O, play me some merry dump to comfort me. + 1. Mus. Not a dump we! 'Tis no time to play now. + + Pet. You will not then? + 1. Mus. No. + + Pet. I will then give it you soundly. + 1. Mus. What will you give us? + + Pet. No money, on my faith, but the gleek. I will give you the + minstrel. + 1. Mus. Then will I give you the serving-creature. + + Pet. Then will I lay the serving-creature's dagger on your pate. + I will carry no crotchets. I'll re you, I'll fa you. Do you + note me? + 1. Mus. An you re us and fa us, you note us. + 2. Mus. Pray you put up your dagger, and put out your wit. + + Pet. Then have at you with my wit! I will dry-beat you with an + iron wit, and put up my iron dagger. Answer me like men. + + 'When griping grief the heart doth wound, + And doleful dumps the mind oppress, + Then music with her silver sound'- + + Why 'silver sound'? Why 'music with her silver sound'? + What say you, Simon Catling? + 1. Mus. Marry, sir, because silver hath a sweet sound. + + Pet. Pretty! What say You, Hugh Rebeck? + 2. Mus. I say 'silver sound' because musicians sound for silver. + + Pet. Pretty too! What say you, James Soundpost? + 3. Mus. Faith, I know not what to say. + + Pet. O, I cry you mercy! you are the singer. I will say for you. It + is 'music with her silver sound' because musicians have no + gold for sounding. + + 'Then music with her silver sound + With speedy help doth lend redress.' [Exit. + + 1. Mus. What a pestilent knave is this same? + 2. Mus. Hang him, Jack! Come, we'll in here, tarry for the + mourners, and stay dinner. + Exeunt. + + + + +ACT V. Scene I. +Mantua. A street. + +Enter Romeo. + + + Rom. If I may trust the flattering truth of sleep + My dreams presage some joyful news at hand. + My bosom's lord sits lightly in his throne, + And all this day an unaccustom'd spirit + Lifts me above the ground with cheerful thoughts. + I dreamt my lady came and found me dead + (Strange dream that gives a dead man leave to think!) + And breath'd such life with kisses in my lips + That I reviv'd and was an emperor. + Ah me! how sweet is love itself possess'd, + When but love's shadows are so rich in joy! + + Enter Romeo's Man Balthasar, booted. + + News from Verona! How now, Balthasar? + Dost thou not bring me letters from the friar? + How doth my lady? Is my father well? + How fares my Juliet? That I ask again, + For nothing can be ill if she be well. + + Man. Then she is well, and nothing can be ill. + Her body sleeps in Capel's monument, + And her immortal part with angels lives. + I saw her laid low in her kindred's vault + And presently took post to tell it you. + O, pardon me for bringing these ill news, + Since you did leave it for my office, sir. + + Rom. Is it e'en so? Then I defy you, stars! + Thou knowest my lodging. Get me ink and paper + And hire posthorses. I will hence to-night. + + Man. I do beseech you, sir, have patience. + Your looks are pale and wild and do import + Some misadventure. + + Rom. Tush, thou art deceiv'd. + Leave me and do the thing I bid thee do. + Hast thou no letters to me from the friar? + + Man. No, my good lord. + + Rom. No matter. Get thee gone + And hire those horses. I'll be with thee straight. + Exit [Balthasar]. + Well, Juliet, I will lie with thee to-night. + Let's see for means. O mischief, thou art swift + To enter in the thoughts of desperate men! + I do remember an apothecary, + And hereabouts 'a dwells, which late I noted + In tatt'red weeds, with overwhelming brows, + Culling of simples. Meagre were his looks, + Sharp misery had worn him to the bones; + And in his needy shop a tortoise hung, + An alligator stuff'd, and other skins + Of ill-shaped fishes; and about his shelves + A beggarly account of empty boxes, + Green earthen pots, bladders, and musty seeds, + Remnants of packthread, and old cakes of roses + Were thinly scattered, to make up a show. + Noting this penury, to myself I said, + 'An if a man did need a poison now + Whose sale is present death in Mantua, + Here lives a caitiff wretch would sell it him.' + O, this same thought did but forerun my need, + And this same needy man must sell it me. + As I remember, this should be the house. + Being holiday, the beggar's shop is shut. What, ho! apothecary! + + Enter Apothecary. + + + Apoth. Who calls so loud? + + Rom. Come hither, man. I see that thou art poor. + Hold, there is forty ducats. Let me have + A dram of poison, such soon-speeding gear + As will disperse itself through all the veins + That the life-weary taker mall fall dead, + And that the trunk may be discharg'd of breath + As violently as hasty powder fir'd + Doth hurry from the fatal cannon's womb. + + Apoth. Such mortal drugs I have; but Mantua's law + Is death to any he that utters them. + + Rom. Art thou so bare and full of wretchedness + And fearest to die? Famine is in thy cheeks, + Need and oppression starveth in thine eyes, + Contempt and beggary hangs upon thy back: + The world is not thy friend, nor the world's law; + The world affords no law to make thee rich; + Then be not poor, but break it and take this. + + Apoth. My poverty but not my will consents. + + Rom. I pay thy poverty and not thy will. + + Apoth. Put this in any liquid thing you will + And drink it off, and if you had the strength + Of twenty men, it would dispatch you straight. + + Rom. There is thy gold- worse poison to men's souls, + Doing more murther in this loathsome world, + Than these poor compounds that thou mayst not sell. + I sell thee poison; thou hast sold me none. + Farewell. Buy food and get thyself in flesh. + Come, cordial and not poison, go with me + To Juliet's grave; for there must I use thee. + Exeunt. + + + + +Scene II. +Verona. Friar Laurence's cell. + +Enter Friar John to Friar Laurence. + + + John. Holy Franciscan friar, brother, ho! + + Enter Friar Laurence. + + + Laur. This same should be the voice of Friar John. + Welcome from Mantua. What says Romeo? + Or, if his mind be writ, give me his letter. + + John. Going to find a barefoot brother out, + One of our order, to associate me + Here in this city visiting the sick, + And finding him, the searchers of the town, + Suspecting that we both were in a house + Where the infectious pestilence did reign, + Seal'd up the doors, and would not let us forth, + So that my speed to Mantua there was stay'd. + + Laur. Who bare my letter, then, to Romeo? + + John. I could not send it- here it is again- + Nor get a messenger to bring it thee, + So fearful were they of infection. + + Laur. Unhappy fortune! By my brotherhood, + The letter was not nice, but full of charge, + Of dear import; and the neglecting it + May do much danger. Friar John, go hence, + Get me an iron crow and bring it straight + Unto my cell. + + John. Brother, I'll go and bring it thee. Exit. + + Laur. Now, must I to the monument alone. + Within this three hours will fair Juliet wake. + She will beshrew me much that Romeo + Hath had no notice of these accidents; + But I will write again to Mantua, + And keep her at my cell till Romeo come- + Poor living corse, clos'd in a dead man's tomb! Exit. + + + + +Scene III. +Verona. A churchyard; in it the monument of the Capulets. + +Enter Paris and his Page with flowers and [a torch]. + + + Par. Give me thy torch, boy. Hence, and stand aloof. + Yet put it out, for I would not be seen. + Under yond yew tree lay thee all along, + Holding thine ear close to the hollow ground. + So shall no foot upon the churchyard tread + (Being loose, unfirm, with digging up of graves) + But thou shalt hear it. Whistle then to me, + As signal that thou hear'st something approach. + Give me those flowers. Do as I bid thee, go. + + Page. [aside] I am almost afraid to stand alone + Here in the churchyard; yet I will adventure. [Retires.] + + Par. Sweet flower, with flowers thy bridal bed I strew + (O woe! thy canopy is dust and stones) + Which with sweet water nightly I will dew; + Or, wanting that, with tears distill'd by moans. + The obsequies that I for thee will keep + Nightly shall be to strew, thy grave and weep. + Whistle Boy. + The boy gives warning something doth approach. + What cursed foot wanders this way to-night + To cross my obsequies and true love's rite? + What, with a torch? Muffle me, night, awhile. [Retires.] + + Enter Romeo, and Balthasar with a torch, a mattock, + and a crow of iron. + + + Rom. Give me that mattock and the wrenching iron. + Hold, take this letter. Early in the morning + See thou deliver it to my lord and father. + Give me the light. Upon thy life I charge thee, + Whate'er thou hearest or seest, stand all aloof + And do not interrupt me in my course. + Why I descend into this bed of death + Is partly to behold my lady's face, + But chiefly to take thence from her dead finger + A precious ring- a ring that I must use + In dear employment. Therefore hence, be gone. + But if thou, jealous, dost return to pry + In what I farther shall intend to do, + By heaven, I will tear thee joint by joint + And strew this hungry churchyard with thy limbs. + The time and my intents are savage-wild, + More fierce and more inexorable far + Than empty tigers or the roaring sea. + + Bal. I will be gone, sir, and not trouble you. + + Rom. So shalt thou show me friendship. Take thou that. + Live, and be prosperous; and farewell, good fellow. + + Bal. [aside] For all this same, I'll hide me hereabout. + His looks I fear, and his intents I doubt. [Retires.] + + Rom. Thou detestable maw, thou womb of death, + Gorg'd with the dearest morsel of the earth, + Thus I enforce thy rotten jaws to open, + And in despite I'll cram thee with more food. + Romeo opens the tomb. + + Par. This is that banish'd haughty Montague + That murd'red my love's cousin- with which grief + It is supposed the fair creature died- + And here is come to do some villanous shame + To the dead bodies. I will apprehend him. + Stop thy unhallowed toil, vile Montague! + Can vengeance be pursu'd further than death? + Condemned villain, I do apprehend thee. + Obey, and go with me; for thou must die. + + Rom. I must indeed; and therefore came I hither. + Good gentle youth, tempt not a desp'rate man. + Fly hence and leave me. Think upon these gone; + Let them affright thee. I beseech thee, youth, + But not another sin upon my head + By urging me to fury. O, be gone! + By heaven, I love thee better than myself, + For I come hither arm'd against myself. + Stay not, be gone. Live, and hereafter say + A madman's mercy bid thee run away. + + Par. I do defy thy, conjuration + And apprehend thee for a felon here. + + Rom. Wilt thou provoke me? Then have at thee, boy! + They fight. + + Page. O Lord, they fight! I will go call the watch. + [Exit. Paris falls.] + + Par. O, I am slain! If thou be merciful, + Open the tomb, lay me with Juliet. [Dies.] + + Rom. In faith, I will. Let me peruse this face. + Mercutio's kinsman, noble County Paris! + What said my man when my betossed soul + Did not attend him as we rode? I think + He told me Paris should have married Juliet. + Said he not so? or did I dream it so? + Or am I mad, hearing him talk of Juliet + To think it was so? O, give me thy hand, + One writ with me in sour misfortune's book! + I'll bury thee in a triumphant grave. + A grave? O, no, a lanthorn, slaught'red youth, + For here lies Juliet, and her beauty makes + This vault a feasting presence full of light. + Death, lie thou there, by a dead man interr'd. + [Lays him in the tomb.] + How oft when men are at the point of death + Have they been merry! which their keepers call + A lightning before death. O, how may I + Call this a lightning? O my love! my wife! + Death, that hath suck'd the honey of thy breath, + Hath had no power yet upon thy beauty. + Thou art not conquer'd. Beauty's ensign yet + Is crimson in thy lips and in thy cheeks, + And death's pale flag is not advanced there. + Tybalt, liest thou there in thy bloody sheet? + O, what more favour can I do to thee + Than with that hand that cut thy youth in twain + To sunder his that was thine enemy? + Forgive me, cousin.' Ah, dear Juliet, + Why art thou yet so fair? Shall I believe + That unsubstantial Death is amorous, + And that the lean abhorred monster keeps + Thee here in dark to be his paramour? + For fear of that I still will stay with thee + And never from this palace of dim night + Depart again. Here, here will I remain + With worms that are thy chambermaids. O, here + Will I set up my everlasting rest + And shake the yoke of inauspicious stars + From this world-wearied flesh. Eyes, look your last! + Arms, take your last embrace! and, lips, O you + The doors of breath, seal with a righteous kiss + A dateless bargain to engrossing death! + Come, bitter conduct; come, unsavoury guide! + Thou desperate pilot, now at once run on + The dashing rocks thy seasick weary bark! + Here's to my love! [Drinks.] O true apothecary! + Thy drugs are quick. Thus with a kiss I die. Falls. + + Enter Friar [Laurence], with lanthorn, crow, and spade. + + + Friar. Saint Francis be my speed! how oft to-night + Have my old feet stumbled at graves! Who's there? + + Bal. Here's one, a friend, and one that knows you well. + + Friar. Bliss be upon you! Tell me, good my friend, + What torch is yond that vainly lends his light + To grubs and eyeless skulls? As I discern, + It burneth in the Capels' monument. + + Bal. It doth so, holy sir; and there's my master, + One that you love. + + Friar. Who is it? + + Bal. Romeo. + + Friar. How long hath he been there? + + Bal. Full half an hour. + + Friar. Go with me to the vault. + + Bal. I dare not, sir. + My master knows not but I am gone hence, + And fearfully did menace me with death + If I did stay to look on his intents. + + Friar. Stay then; I'll go alone. Fear comes upon me. + O, much I fear some ill unthrifty thing. + + Bal. As I did sleep under this yew tree here, + I dreamt my master and another fought, + And that my master slew him. + + Friar. Romeo! + Alack, alack, what blood is this which stains + The stony entrance of this sepulchre? + What mean these masterless and gory swords + To lie discolour'd by this place of peace? [Enters the tomb.] + Romeo! O, pale! Who else? What, Paris too? + And steep'd in blood? Ah, what an unkind hour + Is guilty of this lamentable chance! The lady stirs. + Juliet rises. + + Jul. O comfortable friar! where is my lord? + I do remember well where I should be, + And there I am. Where is my Romeo? + + Friar. I hear some noise. Lady, come from that nest + Of death, contagion, and unnatural sleep. + A greater power than we can contradict + Hath thwarted our intents. Come, come away. + Thy husband in thy bosom there lies dead; + And Paris too. Come, I'll dispose of thee + Among a sisterhood of holy nuns. + Stay not to question, for the watch is coming. + Come, go, good Juliet. I dare no longer stay. + + Jul. Go, get thee hence, for I will not away. + Exit [Friar]. + What's here? A cup, clos'd in my true love's hand? + Poison, I see, hath been his timeless end. + O churl! drunk all, and left no friendly drop + To help me after? I will kiss thy lips. + Haply some poison yet doth hang on them + To make me die with a restorative. [Kisses him.] + Thy lips are warm! + + Chief Watch. [within] Lead, boy. Which way? + Yea, noise? Then I'll be brief. O happy dagger! + [Snatches Romeo's dagger.] + This is thy sheath; there rest, and let me die. + She stabs herself and falls [on Romeo's body]. + + Enter [Paris's] Boy and Watch. + + + Boy. This is the place. There, where the torch doth burn. + + Chief Watch. 'the ground is bloody. Search about the churchyard. + Go, some of you; whoe'er you find attach. + [Exeunt some of the Watch.] + Pitiful sight! here lies the County slain; + And Juliet bleeding, warm, and newly dead, + Who here hath lain this two days buried. + Go, tell the Prince; run to the Capulets; + Raise up the Montagues; some others search. + [Exeunt others of the Watch.] + We see the ground whereon these woes do lie, + But the true ground of all these piteous woes + We cannot without circumstance descry. + + Enter [some of the Watch,] with Romeo's Man [Balthasar]. + + 2. Watch. Here's Romeo's man. We found him in the churchyard. + + Chief Watch. Hold him in safety till the Prince come hither. + + Enter Friar [Laurence] and another Watchman. + + 3. Watch. Here is a friar that trembles, sighs, and weeps. + We took this mattock and this spade from him + As he was coming from this churchyard side. + + Chief Watch. A great suspicion! Stay the friar too. + + Enter the Prince [and Attendants]. + + + Prince. What misadventure is so early up, + That calls our person from our morning rest? + + Enter Capulet and his Wife [with others]. + + + Cap. What should it be, that they so shriek abroad? + + Wife. The people in the street cry 'Romeo,' + Some 'Juliet,' and some 'Paris'; and all run, + With open outcry, toward our monument. + + Prince. What fear is this which startles in our ears? + + Chief Watch. Sovereign, here lies the County Paris slain; + And Romeo dead; and Juliet, dead before, + Warm and new kill'd. + + Prince. Search, seek, and know how this foul murder comes. + + Chief Watch. Here is a friar, and slaughter'd Romeo's man, + With instruments upon them fit to open + These dead men's tombs. + + Cap. O heavens! O wife, look how our daughter bleeds! + This dagger hath mista'en, for, lo, his house + Is empty on the back of Montague, + And it missheathed in my daughter's bosom! + + Wife. O me! this sight of death is as a bell + That warns my old age to a sepulchre. + + Enter Montague [and others]. + + + Prince. Come, Montague; for thou art early up + To see thy son and heir more early down. + + Mon. Alas, my liege, my wife is dead to-night! + Grief of my son's exile hath stopp'd her breath. + What further woe conspires against mine age? + + Prince. Look, and thou shalt see. + + Mon. O thou untaught! what manners is in this, + To press before thy father to a grave? + + Prince. Seal up the mouth of outrage for a while, + Till we can clear these ambiguities + And know their spring, their head, their true descent; + And then will I be general of your woes + And lead you even to death. Meantime forbear, + And let mischance be slave to patience. + Bring forth the parties of suspicion. + + Friar. I am the greatest, able to do least, + Yet most suspected, as the time and place + Doth make against me, of this direful murther; + And here I stand, both to impeach and purge + Myself condemned and myself excus'd. + + Prince. Then say it once what thou dost know in this. + + Friar. I will be brief, for my short date of breath + Is not so long as is a tedious tale. + Romeo, there dead, was husband to that Juliet; + And she, there dead, that Romeo's faithful wife. + I married them; and their stol'n marriage day + Was Tybalt's doomsday, whose untimely death + Banish'd the new-made bridegroom from this city; + For whom, and not for Tybalt, Juliet pin'd. + You, to remove that siege of grief from her, + Betroth'd and would have married her perforce + To County Paris. Then comes she to me + And with wild looks bid me devise some mean + To rid her from this second marriage, + Or in my cell there would she kill herself. + Then gave I her (so tutored by my art) + A sleeping potion; which so took effect + As I intended, for it wrought on her + The form of death. Meantime I writ to Romeo + That he should hither come as this dire night + To help to take her from her borrowed grave, + Being the time the potion's force should cease. + But he which bore my letter, Friar John, + Was stay'd by accident, and yesternight + Return'd my letter back. Then all alone + At the prefixed hour of her waking + Came I to take her from her kindred's vault; + Meaning to keep her closely at my cell + Till I conveniently could send to Romeo. + But when I came, some minute ere the time + Of her awaking, here untimely lay + The noble Paris and true Romeo dead. + She wakes; and I entreated her come forth + And bear this work of heaven with patience; + But then a noise did scare me from the tomb, + And she, too desperate, would not go with me, + But, as it seems, did violence on herself. + All this I know, and to the marriage + Her nurse is privy; and if aught in this + Miscarried by my fault, let my old life + Be sacrific'd, some hour before his time, + Unto the rigour of severest law. + + Prince. We still have known thee for a holy man. + Where's Romeo's man? What can he say in this? + + Bal. I brought my master news of Juliet's death; + And then in post he came from Mantua + To this same place, to this same monument. + This letter he early bid me give his father, + And threat'ned me with death, going in the vault, + If I departed not and left him there. + + Prince. Give me the letter. I will look on it. + Where is the County's page that rais'd the watch? + Sirrah, what made your master in this place? + + Boy. He came with flowers to strew his lady's grave; + And bid me stand aloof, and so I did. + Anon comes one with light to ope the tomb; + And by-and-by my master drew on him; + And then I ran away to call the watch. + + Prince. This letter doth make good the friar's words, + Their course of love, the tidings of her death; + And here he writes that he did buy a poison + Of a poor pothecary, and therewithal + Came to this vault to die, and lie with Juliet. + Where be these enemies? Capulet, Montage, + See what a scourge is laid upon your hate, + That heaven finds means to kill your joys with love! + And I, for winking at you, discords too, + Have lost a brace of kinsmen. All are punish'd. + + Cap. O brother Montague, give me thy hand. + This is my daughter's jointure, for no more + Can I demand. + + Mon. But I can give thee more; + For I will raise her Statue in pure gold, + That whiles Verona by that name is known, + There shall no figure at such rate be set + As that of true and faithful Juliet. + + Cap. As rich shall Romeo's by his lady's lie- + Poor sacrifices of our enmity! + + Prince. A glooming peace this morning with it brings. + The sun for sorrow will not show his head. + Go hence, to have more talk of these sad things; + Some shall be pardon'd, and some punished; + For never was a story of more woe + Than this of Juliet and her Romeo. + Exeunt omnes. + +THE END + + + + + + + + +End of the Project Gutenberg EBook of Romeo and Juliet, by William Shakespeare + +*** END OF THIS PROJECT GUTENBERG EBOOK ROMEO AND JULIET *** + +***** This file should be named 1112.txt or 1112.zip ***** +This and all associated files of various formats will be found in: + http://www.gutenberg.org/1/1/1/1112/ + + + +Updated editions will replace the previous one--the old editions +will be renamed. + +Creating the works from public domain print editions means that no +one owns a United States copyright in these works, so the Foundation +(and you!) can copy and distribute it in the United States without +permission and without paying copyright royalties. Special rules, +set forth in the General Terms of Use part of this license, apply to +copying and distributing Project Gutenberg-tm electronic works to +protect the PROJECT GUTENBERG-tm concept and trademark. Project +Gutenberg is a registered trademark, and may not be used if you +charge for the eBooks, unless you receive specific permission. If you +do not charge anything for copies of this eBook, complying with the +rules is very easy. You may use this eBook for nearly any purpose +such as creation of derivative works, reports, performances and +research. They may be modified and printed and given away--you may do +practically ANYTHING with public domain eBooks. Redistribution is +subject to the trademark license, especially commercial +redistribution. + + + +*** START: FULL LICENSE *** + +THE FULL PROJECT GUTENBERG LICENSE +PLEASE READ THIS BEFORE YOU DISTRIBUTE OR USE THIS WORK + +To protect the Project Gutenberg-tm mission of promoting the free +distribution of electronic works, by using or distributing this work +(or any other work associated in any way with the phrase "Project +Gutenberg"), you agree to comply with all the terms of the Full Project +Gutenberg-tm License (available with this file or online at +http://gutenberg.org/license). + + +Section 1. General Terms of Use and Redistributing Project Gutenberg-tm +electronic works + +1.A. By reading or using any part of this Project Gutenberg-tm +electronic work, you indicate that you have read, understand, agree to +and accept all the terms of this license and intellectual property +(trademark/copyright) agreement. If you do not agree to abide by all +the terms of this agreement, you must cease using and return or destroy +all copies of Project Gutenberg-tm electronic works in your possession. +If you paid a fee for obtaining a copy of or access to a Project +Gutenberg-tm electronic work and you do not agree to be bound by the +terms of this agreement, you may obtain a refund from the person or +entity to whom you paid the fee as set forth in paragraph 1.E.8. + +1.B. "Project Gutenberg" is a registered trademark. It may only be +used on or associated in any way with an electronic work by people who +agree to be bound by the terms of this agreement. There are a few +things that you can do with most Project Gutenberg-tm electronic works +even without complying with the full terms of this agreement. See +paragraph 1.C below. There are a lot of things you can do with Project +Gutenberg-tm electronic works if you follow the terms of this agreement +and help preserve free future access to Project Gutenberg-tm electronic +works. See paragraph 1.E below. + +1.C. The Project Gutenberg Literary Archive Foundation ("the Foundation" +or PGLAF), owns a compilation copyright in the collection of Project +Gutenberg-tm electronic works. Nearly all the individual works in the +collection are in the public domain in the United States. If an +individual work is in the public domain in the United States and you are +located in the United States, we do not claim a right to prevent you from +copying, distributing, performing, displaying or creating derivative +works based on the work as long as all references to Project Gutenberg +are removed. Of course, we hope that you will support the Project +Gutenberg-tm mission of promoting free access to electronic works by +freely sharing Project Gutenberg-tm works in compliance with the terms of +this agreement for keeping the Project Gutenberg-tm name associated with +the work. You can easily comply with the terms of this agreement by +keeping this work in the same format with its attached full Project +Gutenberg-tm License when you share it without charge with others. + +1.D. The copyright laws of the place where you are located also govern +what you can do with this work. Copyright laws in most countries are in +a constant state of change. If you are outside the United States, check +the laws of your country in addition to the terms of this agreement +before downloading, copying, displaying, performing, distributing or +creating derivative works based on this work or any other Project +Gutenberg-tm work. The Foundation makes no representations concerning +the copyright status of any work in any country outside the United +States. + +1.E. Unless you have removed all references to Project Gutenberg: + +1.E.1. The following sentence, with active links to, or other immediate +access to, the full Project Gutenberg-tm License must appear prominently +whenever any copy of a Project Gutenberg-tm work (any work on which the +phrase "Project Gutenberg" appears, or with which the phrase "Project +Gutenberg" is associated) is accessed, displayed, performed, viewed, +copied or distributed: + +This eBook is for the use of anyone anywhere at no cost and with +almost no restrictions whatsoever. You may copy it, give it away or +re-use it under the terms of the Project Gutenberg License included +with this eBook or online at www.gutenberg.org/license + +1.E.2. If an individual Project Gutenberg-tm electronic work is derived +from the public domain (does not contain a notice indicating that it is +posted with permission of the copyright holder), the work can be copied +and distributed to anyone in the United States without paying any fees +or charges. If you are redistributing or providing access to a work +with the phrase "Project Gutenberg" associated with or appearing on the +work, you must comply either with the requirements of paragraphs 1.E.1 +through 1.E.7 or obtain permission for the use of the work and the +Project Gutenberg-tm trademark as set forth in paragraphs 1.E.8 or +1.E.9. + +1.E.3. If an individual Project Gutenberg-tm electronic work is posted +with the permission of the copyright holder, your use and distribution +must comply with both paragraphs 1.E.1 through 1.E.7 and any additional +terms imposed by the copyright holder. Additional terms will be linked +to the Project Gutenberg-tm License for all works posted with the +permission of the copyright holder found at the beginning of this work. + +1.E.4. Do not unlink or detach or remove the full Project Gutenberg-tm +License terms from this work, or any files containing a part of this +work or any other work associated with Project Gutenberg-tm. + +1.E.5. Do not copy, display, perform, distribute or redistribute this +electronic work, or any part of this electronic work, without +prominently displaying the sentence set forth in paragraph 1.E.1 with +active links or immediate access to the full terms of the Project +Gutenberg-tm License. + +1.E.6. You may convert to and distribute this work in any binary, +compressed, marked up, nonproprietary or proprietary form, including any +word processing or hypertext form. However, if you provide access to or +distribute copies of a Project Gutenberg-tm work in a format other than +"Plain Vanilla ASCII" or other format used in the official version +posted on the official Project Gutenberg-tm web site (www.gutenberg.org), +you must, at no additional cost, fee or expense to the user, provide a +copy, a means of exporting a copy, or a means of obtaining a copy upon +request, of the work in its original "Plain Vanilla ASCII" or other +form. Any alternate format must include the full Project Gutenberg-tm +License as specified in paragraph 1.E.1. + +1.E.7. Do not charge a fee for access to, viewing, displaying, +performing, copying or distributing any Project Gutenberg-tm works +unless you comply with paragraph 1.E.8 or 1.E.9. + +1.E.8. You may charge a reasonable fee for copies of or providing +access to or distributing Project Gutenberg-tm electronic works provided +that + +- You pay a royalty fee of 20% of the gross profits you derive from + the use of Project Gutenberg-tm works calculated using the method + you already use to calculate your applicable taxes. The fee is + owed to the owner of the Project Gutenberg-tm trademark, but he + has agreed to donate royalties under this paragraph to the + Project Gutenberg Literary Archive Foundation. Royalty payments + must be paid within 60 days following each date on which you + prepare (or are legally required to prepare) your periodic tax + returns. Royalty payments should be clearly marked as such and + sent to the Project Gutenberg Literary Archive Foundation at the + address specified in Section 4, "Information about donations to + the Project Gutenberg Literary Archive Foundation." + +- You provide a full refund of any money paid by a user who notifies + you in writing (or by e-mail) within 30 days of receipt that s/he + does not agree to the terms of the full Project Gutenberg-tm + License. You must require such a user to return or + destroy all copies of the works possessed in a physical medium + and discontinue all use of and all access to other copies of + Project Gutenberg-tm works. + +- You provide, in accordance with paragraph 1.F.3, a full refund of any + money paid for a work or a replacement copy, if a defect in the + electronic work is discovered and reported to you within 90 days + of receipt of the work. + +- You comply with all other terms of this agreement for free + distribution of Project Gutenberg-tm works. + +1.E.9. If you wish to charge a fee or distribute a Project Gutenberg-tm +electronic work or group of works on different terms than are set +forth in this agreement, you must obtain permission in writing from +both the Project Gutenberg Literary Archive Foundation and Michael +Hart, the owner of the Project Gutenberg-tm trademark. Contact the +Foundation as set forth in Section 3 below. + +1.F. + +1.F.1. Project Gutenberg volunteers and employees expend considerable +effort to identify, do copyright research on, transcribe and proofread +public domain works in creating the Project Gutenberg-tm +collection. Despite these efforts, Project Gutenberg-tm electronic +works, and the medium on which they may be stored, may contain +"Defects," such as, but not limited to, incomplete, inaccurate or +corrupt data, transcription errors, a copyright or other intellectual +property infringement, a defective or damaged disk or other medium, a +computer virus, or computer codes that damage or cannot be read by +your equipment. + +1.F.2. LIMITED WARRANTY, DISCLAIMER OF DAMAGES - Except for the "Right +of Replacement or Refund" described in paragraph 1.F.3, the Project +Gutenberg Literary Archive Foundation, the owner of the Project +Gutenberg-tm trademark, and any other party distributing a Project +Gutenberg-tm electronic work under this agreement, disclaim all +liability to you for damages, costs and expenses, including legal +fees. YOU AGREE THAT YOU HAVE NO REMEDIES FOR NEGLIGENCE, STRICT +LIABILITY, BREACH OF WARRANTY OR BREACH OF CONTRACT EXCEPT THOSE +PROVIDED IN PARAGRAPH 1.F.3. YOU AGREE THAT THE FOUNDATION, THE +TRADEMARK OWNER, AND ANY DISTRIBUTOR UNDER THIS AGREEMENT WILL NOT BE +LIABLE TO YOU FOR ACTUAL, DIRECT, INDIRECT, CONSEQUENTIAL, PUNITIVE OR +INCIDENTAL DAMAGES EVEN IF YOU GIVE NOTICE OF THE POSSIBILITY OF SUCH +DAMAGE. + +1.F.3. LIMITED RIGHT OF REPLACEMENT OR REFUND - If you discover a +defect in this electronic work within 90 days of receiving it, you can +receive a refund of the money (if any) you paid for it by sending a +written explanation to the person you received the work from. If you +received the work on a physical medium, you must return the medium with +your written explanation. The person or entity that provided you with +the defective work may elect to provide a replacement copy in lieu of a +refund. If you received the work electronically, the person or entity +providing it to you may choose to give you a second opportunity to +receive the work electronically in lieu of a refund. If the second copy +is also defective, you may demand a refund in writing without further +opportunities to fix the problem. + +1.F.4. Except for the limited right of replacement or refund set forth +in paragraph 1.F.3, this work is provided to you 'AS-IS' WITH NO OTHER +WARRANTIES OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO +WARRANTIES OF MERCHANTABILITY OR FITNESS FOR ANY PURPOSE. + +1.F.5. Some states do not allow disclaimers of certain implied +warranties or the exclusion or limitation of certain types of damages. +If any disclaimer or limitation set forth in this agreement violates the +law of the state applicable to this agreement, the agreement shall be +interpreted to make the maximum disclaimer or limitation permitted by +the applicable state law. The invalidity or unenforceability of any +provision of this agreement shall not void the remaining provisions. + +1.F.6. INDEMNITY - You agree to indemnify and hold the Foundation, the +trademark owner, any agent or employee of the Foundation, anyone +providing copies of Project Gutenberg-tm electronic works in accordance +with this agreement, and any volunteers associated with the production, +promotion and distribution of Project Gutenberg-tm electronic works, +harmless from all liability, costs and expenses, including legal fees, +that arise directly or indirectly from any of the following which you do +or cause to occur: (a) distribution of this or any Project Gutenberg-tm +work, (b) alteration, modification, or additions or deletions to any +Project Gutenberg-tm work, and (c) any Defect you cause. + + +Section 2. Information about the Mission of Project Gutenberg-tm + +Project Gutenberg-tm is synonymous with the free distribution of +electronic works in formats readable by the widest variety of computers +including obsolete, old, middle-aged and new computers. It exists +because of the efforts of hundreds of volunteers and donations from +people in all walks of life. + +Volunteers and financial support to provide volunteers with the +assistance they need, are critical to reaching Project Gutenberg-tm's +goals and ensuring that the Project Gutenberg-tm collection will +remain freely available for generations to come. In 2001, the Project +Gutenberg Literary Archive Foundation was created to provide a secure +and permanent future for Project Gutenberg-tm and future generations. +To learn more about the Project Gutenberg Literary Archive Foundation +and how your efforts and donations can help, see Sections 3 and 4 +and the Foundation web page at http://www.pglaf.org. + + +Section 3. Information about the Project Gutenberg Literary Archive +Foundation + +The Project Gutenberg Literary Archive Foundation is a non profit +501(c)(3) educational corporation organized under the laws of the +state of Mississippi and granted tax exempt status by the Internal +Revenue Service. The Foundation's EIN or federal tax identification +number is 64-6221541. Its 501(c)(3) letter is posted at +http://pglaf.org/fundraising. Contributions to the Project Gutenberg +Literary Archive Foundation are tax deductible to the full extent +permitted by U.S. federal laws and your state's laws. + +The Foundation's principal office is located at 4557 Melan Dr. S. +Fairbanks, AK, 99712., but its volunteers and employees are scattered +throughout numerous locations. Its business office is located at +809 North 1500 West, Salt Lake City, UT 84116, (801) 596-1887, email +business@pglaf.org. Email contact links and up to date contact +information can be found at the Foundation's web site and official +page at http://pglaf.org + +For additional contact information: + Dr. Gregory B. Newby + Chief Executive and Director + gbnewby@pglaf.org + + +Section 4. Information about Donations to the Project Gutenberg +Literary Archive Foundation + +Project Gutenberg-tm depends upon and cannot survive without wide +spread public support and donations to carry out its mission of +increasing the number of public domain and licensed works that can be +freely distributed in machine readable form accessible by the widest +array of equipment including outdated equipment. Many small donations +($1 to $5,000) are particularly important to maintaining tax exempt +status with the IRS. + +The Foundation is committed to complying with the laws regulating +charities and charitable donations in all 50 states of the United +States. Compliance requirements are not uniform and it takes a +considerable effort, much paperwork and many fees to meet and keep up +with these requirements. We do not solicit donations in locations +where we have not received written confirmation of compliance. To +SEND DONATIONS or determine the status of compliance for any +particular state visit http://pglaf.org + +While we cannot and do not solicit contributions from states where we +have not met the solicitation requirements, we know of no prohibition +against accepting unsolicited donations from donors in such states who +approach us with offers to donate. + +International donations are gratefully accepted, but we cannot make +any statements concerning tax treatment of donations received from +outside the United States. U.S. laws alone swamp our small staff. + +Please check the Project Gutenberg Web pages for current donation +methods and addresses. Donations are accepted in a number of other +ways including checks, online payments and credit card donations. +To donate, please visit: http://pglaf.org/donate + + +Section 5. General Information About Project Gutenberg-tm electronic +works. + +Professor Michael S. Hart is the originator of the Project Gutenberg-tm +concept of a library of electronic works that could be freely shared +with anyone. For thirty years, he produced and distributed Project +Gutenberg-tm eBooks with only a loose network of volunteer support. + + +Project Gutenberg-tm eBooks are often created from several printed +editions, all of which are confirmed as Public Domain in the U.S. +unless a copyright notice is included. Thus, we do not necessarily +keep eBooks in compliance with any particular paper edition. + + +Most people start at our Web site which has the main PG search facility: + + http://www.gutenberg.org + +This Web site includes information about Project Gutenberg-tm, +including how to make donations to the Project Gutenberg Literary +Archive Foundation, how to help produce our new eBooks, and how to +subscribe to our email newsletter to hear about new eBooks. diff --git a/runners/spark/src/test/resources/pg2264.txt b/runners/spark/src/test/resources/pg2264.txt new file mode 100644 index 000000000000..477b8eefedee --- /dev/null +++ b/runners/spark/src/test/resources/pg2264.txt @@ -0,0 +1,3667 @@ +***The Project Gutenberg's Etext of Shakespeare's First Folio*** +********************The Tragedie of Macbeth********************* + +This is our 3rd edition of most of these plays. See the index. + + +Copyright laws are changing all over the world, be sure to check +the copyright laws for your country before posting these files!! + +Please take a look at the important information in this header. +We encourage you to keep this file on your own disk, keeping an +electronic path open for the next readers. Do not remove this. + + +**Welcome To The World of Free Plain Vanilla Electronic Texts** + +**Etexts Readable By Both Humans and By Computers, Since 1971** + +*These Etexts Prepared By Hundreds of Volunteers and Donations* + +Information on contacting Project Gutenberg to get Etexts, and +further information is included below. We need your donations. + + +The Tragedie of Macbeth + +by William Shakespeare + +July, 2000 [Etext #2264] + + +***The Project Gutenberg's Etext of Shakespeare's First Folio*** +********************The Tragedie of Macbeth********************* + +*****This file should be named 0ws3410.txt or 0ws3410.zip****** + +Corrected EDITIONS of our etexts get a new NUMBER, 0ws3411.txt +VERSIONS based on separate sources get new LETTER, 0ws3410a.txt + + +Project Gutenberg Etexts are usually created from multiple editions, +all of which are in the Public Domain in the United States, unless a +copyright notice is included. Therefore, we usually do NOT keep any +of these books in compliance with any particular paper edition. + + +We are now trying to release all our books one month in advance +of the official release dates, leaving time for better editing. + +Please note: neither this list nor its contents are final till +midnight of the last day of the month of any such announcement. +The official release date of all Project Gutenberg Etexts is at +Midnight, Central Time, of the last day of the stated month. A +preliminary version may often be posted for suggestion, comment +and editing by those who wish to do so. To be sure you have an +up to date first edition [xxxxx10x.xxx] please check file sizes +in the first week of the next month. Since our ftp program has +a bug in it that scrambles the date [tried to fix and failed] a +look at the file size will have to do, but we will try to see a +new copy has at least one byte more or less. + + +Information about Project Gutenberg (one page) + +We produce about two million dollars for each hour we work. The +time it takes us, a rather conservative estimate, is fifty hours +to get any etext selected, entered, proofread, edited, copyright +searched and analyzed, the copyright letters written, etc. This +projected audience is one hundred million readers. If our value +per text is nominally estimated at one dollar then we produce $2 +million dollars per hour this year as we release thirty-six text +files per month, or 432 more Etexts in 1999 for a total of 2000+ +If these reach just 10% of the computerized population, then the +total should reach over 200 billion Etexts given away this year. + +The Goal of Project Gutenberg is to Give Away One Trillion Etext +Files by December 31, 2001. [10,000 x 100,000,000 = 1 Trillion] +This is ten thousand titles each to one hundred million readers, +which is only ~5% of the present number of computer users. + +At our revised rates of production, we will reach only one-third +of that goal by the end of 2001, or about 3,333 Etexts unless we +manage to get some real funding; currently our funding is mostly +from Michael Hart's salary at Carnegie-Mellon University, and an +assortment of sporadic gifts; this salary is only good for a few +more years, so we are looking for something to replace it, as we +don't want Project Gutenberg to be so dependent on one person. + +We need your donations more than ever! + + +All donations should be made to "Project Gutenberg/CMU": and are +tax deductible to the extent allowable by law. (CMU = Carnegie- +Mellon University). + +For these and other matters, please mail to: + +Project Gutenberg +P. O. Box 2782 +Champaign, IL 61825 + +When all other email fails. . .try our Executive Director: +Michael S. Hart +hart@pobox.com forwards to hart@prairienet.org and archive.org +if your mail bounces from archive.org, I will still see it, if +it bounces from prairienet.org, better resend later on. . . . + +We would prefer to send you this information by email. + +****** + +To access Project Gutenberg etexts, use any Web browser +to view http://promo.net/pg. This site lists Etexts by +author and by title, and includes information about how +to get involved with Project Gutenberg. You could also +download our past Newsletters, or subscribe here. This +is one of our major sites, please email hart@pobox.com, +for a more complete list of our various sites. + +To go directly to the etext collections, use FTP or any +Web browser to visit a Project Gutenberg mirror (mirror +sites are available on 7 continents; mirrors are listed +at http://promo.net/pg). + +Mac users, do NOT point and click, typing works better. + +Example FTP session: + +ftp sunsite.unc.edu +login: anonymous +password: your@login +cd pub/docs/books/gutenberg +cd etext90 through etext99 +dir [to see files] +get or mget [to get files. . .set bin for zip files] +GET GUTINDEX.?? [to get a year's listing of books, e.g., GUTINDEX.99] +GET GUTINDEX.ALL [to get a listing of ALL books] + +*** + +**Information prepared by the Project Gutenberg legal advisor** + +(Three Pages) + + +***START**THE SMALL PRINT!**FOR PUBLIC DOMAIN ETEXTS**START*** +Why is this "Small Print!" statement here? You know: lawyers. +They tell us you might sue us if there is something wrong with +your copy of this etext, even if you got it for free from +someone other than us, and even if what's wrong is not our +fault. So, among other things, this "Small Print!" statement +disclaims most of our liability to you. It also tells you how +you can distribute copies of this etext if you want to. + +*BEFORE!* YOU USE OR READ THIS ETEXT +By using or reading any part of this PROJECT GUTENBERG-tm +etext, you indicate that you understand, agree to and accept +this "Small Print!" statement. If you do not, you can receive +a refund of the money (if any) you paid for this etext by +sending a request within 30 days of receiving it to the person +you got it from. If you received this etext on a physical +medium (such as a disk), you must return it with your request. + +ABOUT PROJECT GUTENBERG-TM ETEXTS +This PROJECT GUTENBERG-tm etext, like most PROJECT GUTENBERG- +tm etexts, is a "public domain" work distributed by Professor +Michael S. Hart through the Project Gutenberg Association at +Carnegie-Mellon University (the "Project"). Among other +things, this means that no one owns a United States copyright +on or for this work, so the Project (and you!) can copy and +distribute it in the United States without permission and +without paying copyright royalties. Special rules, set forth +below, apply if you wish to copy and distribute this etext +under the Project's "PROJECT GUTENBERG" trademark. + +To create these etexts, the Project expends considerable +efforts to identify, transcribe and proofread public domain +works. Despite these efforts, the Project's etexts and any +medium they may be on may contain "Defects". Among other +things, Defects may take the form of incomplete, inaccurate or +corrupt data, transcription errors, a copyright or other +intellectual property infringement, a defective or damaged +disk or other etext medium, a computer virus, or computer +codes that damage or cannot be read by your equipment. + +LIMITED WARRANTY; DISCLAIMER OF DAMAGES +But for the "Right of Replacement or Refund" described below, +[1] the Project (and any other party you may receive this +etext from as a PROJECT GUTENBERG-tm etext) disclaims all +liability to you for damages, costs and expenses, including +legal fees, and [2] YOU HAVE NO REMEDIES FOR NEGLIGENCE OR +UNDER STRICT LIABILITY, OR FOR BREACH OF WARRANTY OR CONTRACT, +INCLUDING BUT NOT LIMITED TO INDIRECT, CONSEQUENTIAL, PUNITIVE +OR INCIDENTAL DAMAGES, EVEN IF YOU GIVE NOTICE OF THE +POSSIBILITY OF SUCH DAMAGES. + +If you discover a Defect in this etext within 90 days of +receiving it, you can receive a refund of the money (if any) +you paid for it by sending an explanatory note within that +time to the person you received it from. If you received it +on a physical medium, you must return it with your note, and +such person may choose to alternatively give you a replacement +copy. If you received it electronically, such person may +choose to alternatively give you a second opportunity to +receive it electronically. + +THIS ETEXT IS OTHERWISE PROVIDED TO YOU "AS-IS". NO OTHER +WARRANTIES OF ANY KIND, EXPRESS OR IMPLIED, ARE MADE TO YOU AS +TO THE ETEXT OR ANY MEDIUM IT MAY BE ON, INCLUDING BUT NOT +LIMITED TO WARRANTIES OF MERCHANTABILITY OR FITNESS FOR A +PARTICULAR PURPOSE. + +Some states do not allow disclaimers of implied warranties or +the exclusion or limitation of consequential damages, so the +above disclaimers and exclusions may not apply to you, and you +may have other legal rights. + +INDEMNITY +You will indemnify and hold the Project, its directors, +officers, members and agents harmless from all liability, cost +and expense, including legal fees, that arise directly or +indirectly from any of the following that you do or cause: +[1] distribution of this etext, [2] alteration, modification, +or addition to the etext, or [3] any Defect. + +DISTRIBUTION UNDER "PROJECT GUTENBERG-tm" +You may distribute copies of this etext electronically, or by +disk, book or any other medium if you either delete this +"Small Print!" and all other references to Project Gutenberg, +or: + +[1] Only give exact copies of it. Among other things, this + requires that you do not remove, alter or modify the + etext or this "small print!" statement. You may however, + if you wish, distribute this etext in machine readable + binary, compressed, mark-up, or proprietary form, + including any form resulting from conversion by word pro- + cessing or hypertext software, but only so long as + *EITHER*: + + [*] The etext, when displayed, is clearly readable, and + does *not* contain characters other than those + intended by the author of the work, although tilde + (~), asterisk (*) and underline (_) characters may + be used to convey punctuation intended by the + author, and additional characters may be used to + indicate hypertext links; OR + + [*] The etext may be readily converted by the reader at + no expense into plain ASCII, EBCDIC or equivalent + form by the program that displays the etext (as is + the case, for instance, with most word processors); + OR + + [*] You provide, or agree to also provide on request at + no additional cost, fee or expense, a copy of the + etext in its original plain ASCII form (or in EBCDIC + or other equivalent proprietary form). + +[2] Honor the etext refund and replacement provisions of this + "Small Print!" statement. + +[3] Pay a trademark license fee to the Project of 20% of the + net profits you derive calculated using the method you + already use to calculate your applicable taxes. If you + don't derive profits, no royalty is due. Royalties are + payable to "Project Gutenberg Association/Carnegie-Mellon + University" within the 60 days following each + date you prepare (or were legally required to prepare) + your annual (or equivalent periodic) tax return. + +WHAT IF YOU *WANT* TO SEND MONEY EVEN IF YOU DON'T HAVE TO? +The Project gratefully accepts contributions in money, time, +scanning machines, OCR software, public domain etexts, royalty +free copyright licenses, and every other sort of contribution +you can think of. Money should be paid to "Project Gutenberg +Association / Carnegie-Mellon University". + +*END*THE SMALL PRINT! FOR PUBLIC DOMAIN ETEXTS*Ver.04.29.93*END* + + + + + +Project Gutenberg's Etext of Shakespeare's The Tragedie of Macbeth + + + + + +Executive Director's Notes: + +In addition to the notes below, and so you will *NOT* think all +the spelling errors introduced by the printers of the time have +been corrected, here are the first few lines of Hamlet, as they +are presented herein: + + Barnardo. Who's there? + Fran. Nay answer me: Stand & vnfold +your selfe + + Bar. Long liue the King + +*** + +As I understand it, the printers often ran out of certain words +or letters they had often packed into a "cliche". . .this is the +original meaning of the term cliche. . .and thus, being unwilling +to unpack the cliches, and thus you will see some substitutions +that look very odd. . .such as the exchanges of u for v, v for u, +above. . .and you may wonder why they did it this way, presuming +Shakespeare did not actually write the play in this manner. . . . + +The answer is that they MAY have packed "liue" into a cliche at a +time when they were out of "v"'s. . .possibly having used "vv" in +place of some "w"'s, etc. This was a common practice of the day, +as print was still quite expensive, and they didn't want to spend +more on a wider selection of characters than they had to. + +You will find a lot of these kinds of "errors" in this text, as I +have mentioned in other times and places, many "scholars" have an +extreme attachment to these errors, and many have accorded them a +very high place in the "canon" of Shakespeare. My father read an +assortment of these made available to him by Cambridge University +in England for several months in a glass room constructed for the +purpose. To the best of my knowledge he read ALL those available +. . .in great detail. . .and determined from the various changes, +that Shakespeare most likely did not write in nearly as many of a +variety of errors we credit him for, even though he was in/famous +for signing his name with several different spellings. + +So, please take this into account when reading the comments below +made by our volunteer who prepared this file: you may see errors +that are "not" errors. . . . + +So. . .with this caveat. . .we have NOT changed the canon errors, +here is the Project Gutenberg Etext of Shakespeare's The Tragedie +of Macbeth. + +Michael S. Hart +Project Gutenberg +Executive Director + + +*** + + +Scanner's Notes: What this is and isn't. This was taken from +a copy of Shakespeare's first folio and it is as close as I can +come in ASCII to the printed text. + +The elongated S's have been changed to small s's and the +conjoined ae have been changed to ae. I have left the spelling, +punctuation, capitalization as close as possible to the +printed text. I have corrected some spelling mistakes (I have put +together a spelling dictionary devised from the spellings of the +Geneva Bible and Shakespeare's First Folio and have unified +spellings according to this template), typo's and expanded +abbreviations as I have come across them. Everything within +brackets [] is what I have added. So if you don't like that +you can delete everything within the brackets if you want a +purer Shakespeare. + +Another thing that you should be aware of is that there are textual +differences between various copies of the first folio. So there may +be differences (other than what I have mentioned above) between +this and other first folio editions. This is due to the printer's +habit of setting the type and running off a number of copies and +then proofing the printed copy and correcting the type and then +continuing the printing run. The proof run wasn't thrown away but +incorporated into the printed copies. This is just the way it is. +The text I have used was a composite of more than 30 different +First Folio editions' best pages. + +If you find any scanning errors, out and out typos, punctuation +errors, or if you disagree with my spelling choices please feel +free to email me those errors. I wish to make this the best +etext possible. My email address for right now are haradda@aol.com +and davidr@inconnect.com. I hope that you enjoy this. + +David Reed + +The Tragedie of Macbeth + +Actus Primus. Scoena Prima. + +Thunder and Lightning. Enter three Witches. + + 1. When shall we three meet againe? +In Thunder, Lightning, or in Raine? + 2. When the Hurley-burley's done, +When the Battaile's lost, and wonne + + 3. That will be ere the set of Sunne + + 1. Where the place? + 2. Vpon the Heath + + 3. There to meet with Macbeth + + 1. I come, Gray-Malkin + + All. Padock calls anon: faire is foule, and foule is faire, +Houer through the fogge and filthie ayre. + +Exeunt. + + +Scena Secunda. + +Alarum within. Enter King Malcome, Donalbaine, Lenox, with +attendants, +meeting a bleeding Captaine. + + King. What bloody man is that? he can report, +As seemeth by his plight, of the Reuolt +The newest state + + Mal. This is the Serieant, +Who like a good and hardie Souldier fought +'Gainst my Captiuitie: Haile braue friend; +Say to the King, the knowledge of the Broyle, +As thou didst leaue it + + Cap. Doubtfull it stood, +As two spent Swimmers, that doe cling together, +And choake their Art: The mercilesse Macdonwald +(Worthie to be a Rebell, for to that +The multiplying Villanies of Nature +Doe swarme vpon him) from the Westerne Isles +Of Kernes and Gallowgrosses is supply'd, +And Fortune on his damned Quarry smiling, +Shew'd like a Rebells Whore: but all's too weake: +For braue Macbeth (well hee deserues that Name) +Disdayning Fortune, with his brandisht Steele, +Which smoak'd with bloody execution +(Like Valours Minion) caru'd out his passage, +Till hee fac'd the Slaue: +Which neu'r shooke hands, nor bad farwell to him, +Till he vnseam'd him from the Naue toth' Chops, +And fix'd his Head vpon our Battlements + + King. O valiant Cousin, worthy Gentleman + + Cap. As whence the Sunne 'gins his reflection, +Shipwracking Stormes, and direfull Thunders: +So from that Spring, whence comfort seem'd to come, +Discomfort swells: Marke King of Scotland, marke, +No sooner Iustice had, with Valour arm'd, +Compell'd these skipping Kernes to trust their heeles, +But the Norweyan Lord, surueying vantage, +With furbusht Armes, and new supplyes of men, +Began a fresh assault + + King. Dismay'd not this our Captaines, Macbeth and +Banquoh? + Cap. Yes, as Sparrowes, Eagles; +Or the Hare, the Lyon: +If I say sooth, I must report they were +As Cannons ouer-charg'd with double Cracks, +So they doubly redoubled stroakes vpon the Foe: +Except they meant to bathe in reeking Wounds, +Or memorize another Golgotha, +I cannot tell: but I am faint, +My Gashes cry for helpe + + King. So well thy words become thee, as thy wounds, +They smack of Honor both: Goe get him Surgeons. +Enter Rosse and Angus. + +Who comes here? + Mal. The worthy Thane of Rosse + + Lenox. What a haste lookes through his eyes? +So should he looke, that seemes to speake things strange + + Rosse. God saue the King + + King. Whence cam'st thou, worthy Thane? + Rosse. From Fiffe, great King, +Where the Norweyan Banners flowt the Skie, +And fanne our people cold. +Norway himselfe, with terrible numbers, +Assisted by that most disloyall Traytor, +The Thane of Cawdor, began a dismall Conflict, +Till that Bellona's Bridegroome, lapt in proofe, +Confronted him with selfe-comparisons, +Point against Point, rebellious Arme 'gainst Arme, +Curbing his lauish spirit: and to conclude, +The Victorie fell on vs + + King. Great happinesse + + Rosse. That now Sweno, the Norwayes King, +Craues composition: +Nor would we deigne him buriall of his men, +Till he disbursed, at Saint Colmes ynch, +Ten thousand Dollars, to our generall vse + + King. No more that Thane of Cawdor shall deceiue +Our Bosome interest: Goe pronounce his present death, +And with his former Title greet Macbeth + + Rosse. Ile see it done + + King. What he hath lost, Noble Macbeth hath wonne. + +Exeunt. + + +Scena Tertia. + +Thunder. Enter the three Witches. + + 1. Where hast thou beene, Sister? + 2. Killing Swine + + 3. Sister, where thou? + 1. A Saylors Wife had Chestnuts in her Lappe, +And mouncht, & mouncht, and mouncht: +Giue me, quoth I. +Aroynt thee, Witch, the rumpe-fed Ronyon cryes. +Her Husband's to Aleppo gone, Master o'th' Tiger: +But in a Syue Ile thither sayle, +And like a Rat without a tayle, +Ile doe, Ile doe, and Ile doe + + 2. Ile giue thee a Winde + + 1. Th'art kinde + + 3. And I another + + 1. I my selfe haue all the other, +And the very Ports they blow, +All the Quarters that they know, +I'th' Ship-mans Card. +Ile dreyne him drie as Hay: +Sleepe shall neyther Night nor Day +Hang vpon his Pent-house Lid: +He shall liue a man forbid: +Wearie Seu'nights, nine times nine, +Shall he dwindle, peake, and pine: +Though his Barke cannot be lost, +Yet it shall be Tempest-tost. +Looke what I haue + + 2. Shew me, shew me + + 1. Here I haue a Pilots Thumbe, +Wrackt, as homeward he did come. + +Drum within. + + 3. A Drumme, a Drumme: +Macbeth doth come + + All. The weyward Sisters, hand in hand, +Posters of the Sea and Land, +Thus doe goe, about, about, +Thrice to thine, and thrice to mine, +And thrice againe, to make vp nine. +Peace, the Charme's wound vp. +Enter Macbeth and Banquo. + + Macb. So foule and faire a day I haue not seene + + Banquo. How farre is't call'd to Soris? What are these, +So wither'd, and so wilde in their attyre, +That looke not like th' Inhabitants o'th' Earth, +And yet are on't? Liue you, or are you aught +That man may question? you seeme to vnderstand me, +By each at once her choppie finger laying +Vpon her skinnie Lips: you should be Women, +And yet your Beards forbid me to interprete +That you are so + + Mac. Speake if you can: what are you? + 1. All haile Macbeth, haile to thee Thane of Glamis + + 2. All haile Macbeth, haile to thee Thane of Cawdor + + 3. All haile Macbeth, that shalt be King hereafter + + Banq. Good Sir, why doe you start, and seeme to feare +Things that doe sound so faire? i'th' name of truth +Are ye fantasticall, or that indeed +Which outwardly ye shew? My Noble Partner +You greet with present Grace, and great prediction +Of Noble hauing, and of Royall hope, +That he seemes wrapt withall: to me you speake not. +If you can looke into the Seedes of Time, +And say, which Graine will grow, and which will not, +Speake then to me, who neyther begge, nor feare +Your fauors, nor your hate + + 1. Hayle + + 2. Hayle + + 3. Hayle + + 1. Lesser than Macbeth, and greater + + 2. Not so happy, yet much happyer + + 3. Thou shalt get Kings, though thou be none: +So all haile Macbeth, and Banquo + + 1. Banquo, and Macbeth, all haile + + Macb. Stay you imperfect Speakers, tell me more: +By Sinells death, I know I am Thane of Glamis, +But how, of Cawdor? the Thane of Cawdor liues +A prosperous Gentleman: And to be King, +Stands not within the prospect of beleefe, +No more then to be Cawdor. Say from whence +You owe this strange Intelligence, or why +Vpon this blasted Heath you stop our way +With such Prophetique greeting? +Speake, I charge you. + +Witches vanish. + + Banq. The Earth hath bubbles, as the Water ha's, +And these are of them: whither are they vanish'd? + Macb. Into the Ayre: and what seem'd corporall, +Melted, as breath into the Winde. +Would they had stay'd + + Banq. Were such things here, as we doe speake about? +Or haue we eaten on the insane Root, +That takes the Reason Prisoner? + Macb. Your Children shall be Kings + + Banq. You shall be King + + Macb. And Thane of Cawdor too: went it not so? + Banq. Toth' selfe-same tune and words: who's here? +Enter Rosse and Angus. + + Rosse. The King hath happily receiu'd, Macbeth, +The newes of thy successe: and when he reades +Thy personall Venture in the Rebels sight, +His Wonders and his Prayses doe contend, +Which should be thine, or his: silenc'd with that, +In viewing o're the rest o'th' selfe-same day, +He findes thee in the stout Norweyan Rankes, +Nothing afeard of what thy selfe didst make +Strange Images of death, as thick as Tale +Can post with post, and euery one did beare +Thy prayses in his Kingdomes great defence, +And powr'd them downe before him + + Ang. Wee are sent, +To giue thee from our Royall Master thanks, +Onely to harrold thee into his sight, +Not pay thee + + Rosse. And for an earnest of a greater Honor, +He bad me, from him, call thee Thane of Cawdor: +In which addition, haile most worthy Thane, +For it is thine + + Banq. What, can the Deuill speake true? + Macb. The Thane of Cawdor liues: +Why doe you dresse me in borrowed Robes? + Ang. Who was the Thane, liues yet, +But vnder heauie Iudgement beares that Life, +Which he deserues to loose. +Whether he was combin'd with those of Norway, +Or did lyne the Rebell with hidden helpe, +And vantage; or that with both he labour'd +In his Countreyes wracke, I know not: +But Treasons Capitall, confess'd, and prou'd, +Haue ouerthrowne him + + Macb. Glamys, and Thane of Cawdor: +The greatest is behinde. Thankes for your paines. +Doe you not hope your Children shall be Kings, +When those that gaue the Thane of Cawdor to me, +Promis'd no lesse to them + + Banq. That trusted home, +Might yet enkindle you vnto the Crowne, +Besides the Thane of Cawdor. But 'tis strange: +And oftentimes, to winne vs to our harme, +The Instruments of Darknesse tell vs Truths, +Winne vs with honest Trifles, to betray's +In deepest consequence. +Cousins, a word, I pray you + + Macb. Two Truths are told, +As happy Prologues to the swelling Act +Of the Imperiall Theame. I thanke you Gentlemen: +This supernaturall solliciting +Cannot be ill; cannot be good. +If ill? why hath it giuen me earnest of successe, +Commencing in a Truth? I am Thane of Cawdor. +If good? why doe I yeeld to that suggestion, +Whose horrid Image doth vnfixe my Heire, +And make my seated Heart knock at my Ribbes, +Against the vse of Nature? Present Feares +Are lesse then horrible Imaginings: +My Thought, whose Murther yet is but fantasticall, +Shakes so my single state of Man, +That Function is smother'd in surmise, +And nothing is, but what is not + + Banq. Looke how our Partner's rapt + + Macb. If Chance will haue me King, +Why Chance may Crowne me, +Without my stirre + + Banq. New Honors come vpon him +Like our strange Garments, cleaue not to their mould, +But with the aid of vse + + Macb. Come what come may, +Time, and the Houre, runs through the roughest Day + + Banq. Worthy Macbeth, wee stay vpon your leysure + + Macb. Giue me your fauour: +My dull Braine was wrought with things forgotten. +Kinde Gentlemen, your paines are registred, +Where euery day I turne the Leafe, +To reade them. +Let vs toward the King: thinke vpon +What hath chanc'd: and at more time, +The Interim hauing weigh'd it, let vs speake +Our free Hearts each to other + + Banq. Very gladly + + Macb. Till then enough: +Come friends. + +Exeunt. + + +Scena Quarta. + +Flourish. Enter King, Lenox, Malcolme, Donalbaine, and +Attendants. + + King. Is execution done on Cawdor? +Or not those in Commission yet return'd? + Mal. My Liege, they are not yet come back. +But I haue spoke with one that saw him die: +Who did report, that very frankly hee +Confess'd his Treasons, implor'd your Highnesse Pardon, +And set forth a deepe Repentance: +Nothing in his Life became him, +Like the leauing it. Hee dy'de, +As one that had beene studied in his death, +To throw away the dearest thing he ow'd, +As 'twere a carelesse Trifle + + King. There's no Art, +To finde the Mindes construction in the Face. +He was a Gentleman, on whom I built +An absolute Trust. +Enter Macbeth, Banquo, Rosse, and Angus. + +O worthyest Cousin, +The sinne of my Ingratitude euen now +Was heauie on me. Thou art so farre before, +That swiftest Wing of Recompence is slow, +To ouertake thee. Would thou hadst lesse deseru'd, +That the proportion both of thanks, and payment, +Might haue beene mine: onely I haue left to say, +More is thy due, then more then all can pay + + Macb. The seruice, and the loyaltie I owe, +In doing it, payes it selfe. +Your Highnesse part, is to receiue our Duties: +And our Duties are to your Throne, and State, +Children, and Seruants; which doe but what they should, +By doing euery thing safe toward your Loue +And Honor + + King. Welcome hither: +I haue begun to plant thee, and will labour +To make thee full of growing. Noble Banquo, +That hast no lesse deseru'd, nor must be knowne +No lesse to haue done so: Let me enfold thee, +And hold thee to my Heart + + Banq. There if I grow, +The Haruest is your owne + + King. My plenteous Ioyes, +Wanton in fulnesse, seeke to hide themselues +In drops of sorrow. Sonnes, Kinsmen, Thanes, +And you whose places are the nearest, know, +We will establish our Estate vpon +Our eldest, Malcolme, whom we name hereafter, +The Prince of Cumberland: which Honor must +Not vnaccompanied, inuest him onely, +But signes of Noblenesse, like Starres, shall shine +On all deseruers. From hence to Envernes, +And binde vs further to you + + Macb. The Rest is Labor, which is not vs'd for you: +Ile be my selfe the Herbenger, and make ioyfull +The hearing of my Wife, with your approach: +So humbly take my leaue + + King. My worthy Cawdor + + Macb. The Prince of Cumberland: that is a step, +On which I must fall downe, or else o're-leape, +For in my way it lyes. Starres hide your fires, +Let not Light see my black and deepe desires: +The Eye winke at the Hand: yet let that bee, +Which the Eye feares, when it is done to see. +Enter. + + King. True worthy Banquo: he is full so valiant, +And in his commendations, I am fed: +It is a Banquet to me. Let's after him, +Whose care is gone before, to bid vs welcome: +It is a peerelesse Kinsman. + +Flourish. Exeunt. + + +Scena Quinta. + +Enter Macbeths Wife alone with a Letter. + + Lady. They met me in the day of successe: and I haue +learn'd by the perfect'st report, they haue more in them, then +mortall knowledge. When I burnt in desire to question them +further, they made themselues Ayre, into which they vanish'd. +Whiles I stood rapt in the wonder of it, came Missiues from +the King, who all-hail'd me Thane of Cawdor, by which Title +before, these weyward Sisters saluted me, and referr'd me to +the comming on of time, with haile King that shalt be. This +haue I thought good to deliuer thee (my dearest Partner of +Greatnesse) that thou might'st not loose the dues of reioycing +by being ignorant of what Greatnesse is promis'd thee. Lay +it to thy heart and farewell. +Glamys thou art, and Cawdor, and shalt be +What thou art promis'd: yet doe I feare thy Nature, +It is too full o'th' Milke of humane kindnesse, +To catch the neerest way. Thou would'st be great, +Art not without Ambition, but without +The illnesse should attend it. What thou would'st highly, +That would'st thou holily: would'st not play false, +And yet would'st wrongly winne. +Thould'st haue, great Glamys, that which cryes, +Thus thou must doe, if thou haue it; +And that which rather thou do'st feare to doe, +Then wishest should be vndone. High thee hither, +That I may powre my Spirits in thine Eare, +And chastise with the valour of my Tongue +All that impeides thee from the Golden Round, +Which Fate and Metaphysicall ayde doth seeme +To haue thee crown'd withall. +Enter Messenger. + +What is your tidings? + Mess. The King comes here to Night + + Lady. Thou'rt mad to say it. +Is not thy Master with him? who, wer't so, +Would haue inform'd for preparation + + Mess. So please you, it is true: our Thane is comming: +One of my fellowes had the speed of him; +Who almost dead for breath, had scarcely more +Then would make vp his Message + + Lady. Giue him tending, +He brings great newes, + +Exit Messenger. + +The Rauen himselfe is hoarse, +That croakes the fatall entrance of Duncan +Vnder my Battlements. Come you Spirits, +That tend on mortall thoughts, vnsex me here, +And fill me from the Crowne to the Toe, top-full +Of direst Crueltie: make thick my blood, +Stop vp th' accesse, and passage to Remorse, +That no compunctious visitings of Nature +Shake my fell purpose, nor keepe peace betweene +Th' effect, and hit. Come to my Womans Brests, +And take my Milke for Gall, you murth'ring Ministers, +Where-euer, in your sightlesse substances, +You wait on Natures Mischiefe. Come thick Night, +And pall thee in the dunnest smoake of Hell, + +That my keene Knife see not the Wound it makes, +Nor Heauen peepe through the Blanket of the darke, +To cry, hold, hold. +Enter Macbeth. + +Great Glamys, worthy Cawdor, +Greater then both, by the all-haile hereafter, +Thy Letters haue transported me beyond +This ignorant present, and I feele now +The future in the instant + + Macb. My dearest Loue, +Duncan comes here to Night + + Lady. And when goes hence? + Macb. To morrow, as he purposes + + Lady. O neuer, +Shall Sunne that Morrow see. +Your Face, my Thane, is as a Booke, where men +May reade strange matters, to beguile the time. +Looke like the time, beare welcome in your Eye, +Your Hand, your Tongue: looke like th' innocent flower, +But be the Serpent vnder't. He that's comming, +Must be prouided for: and you shall put +This Nights great Businesse into my dispatch, +Which shall to all our Nights, and Dayes to come, +Giue solely soueraigne sway, and Masterdome + + Macb. We will speake further, + Lady. Onely looke vp cleare: +To alter fauor, euer is to feare: +Leaue all the rest to me. + +Exeunt. + + +Scena Sexta. + +Hoboyes, and Torches. Enter King, Malcolme, Donalbaine, +Banquo, Lenox, +Macduff, Rosse, Angus, and Attendants. + + King. This Castle hath a pleasant seat, +The ayre nimbly and sweetly recommends it selfe +Vnto our gentle sences + + Banq. This Guest of Summer, +The Temple-haunting Barlet does approue, +By his loued Mansonry, that the Heauens breath +Smells wooingly here: no Iutty frieze, +Buttrice, nor Coigne of Vantage, but this Bird +Hath made his pendant Bed, and procreant Cradle, +Where they must breed, and haunt: I haue obseru'd +The ayre is delicate. +Enter Lady. + + King. See, see our honor'd Hostesse: +The Loue that followes vs, sometime is our trouble, +Which still we thanke as Loue. Herein I teach you, +How you shall bid God-eyld vs for your paines, +And thanke vs for your trouble + + Lady. All our seruice, +In euery point twice done, and then done double, +Were poore, and single Businesse, to contend +Against those Honors deepe, and broad, +Wherewith your Maiestie loades our House: +For those of old, and the late Dignities, +Heap'd vp to them, we rest your Ermites + + King. Where's the Thane of Cawdor? +We courst him at the heeles, and had a purpose +To be his Purueyor: But he rides well, +And his great Loue (sharpe as his Spurre) hath holp him +To his home before vs: Faire and Noble Hostesse +We are your guest to night + + La. Your Seruants euer, +Haue theirs, themselues, and what is theirs in compt, +To make their Audit at your Highnesse pleasure, +Still to returne your owne + + King. Giue me your hand: +Conduct me to mine Host we loue him highly, +And shall continue, our Graces towards him. +By your leaue Hostesse. + +Exeunt. + +Scena Septima. + +Hoboyes. Torches. Enter a Sewer, and diuers Seruants with Dishes +and +Seruice ouer the Stage. Then enter Macbeth + + Macb. If it were done, when 'tis done, then 'twer well, +It were done quickly: If th' Assassination +Could trammell vp the Consequence, and catch +With his surcease, Successe: that but this blow +Might be the be all, and the end all. Heere, +But heere, vpon this Banke and Schoole of time, +Wee'ld iumpe the life to come. But in these Cases, +We still haue iudgement heere, that we but teach +Bloody Instructions, which being taught, returne +To plague th' Inuenter, this euen-handed Iustice +Commends th' Ingredience of our poyson'd Challice +To our owne lips. Hee's heere in double trust; +First, as I am his Kinsman, and his Subiect, +Strong both against the Deed: Then, as his Host, +Who should against his Murtherer shut the doore, +Not beare the knife my selfe. Besides, this Duncane +Hath borne his Faculties so meeke; hath bin +So cleere in his great Office, that his Vertues +Will pleade like Angels, Trumpet-tongu'd against +The deepe damnation of his taking off: +And Pitty, like a naked New-borne-Babe, +Striding the blast, or Heauens Cherubin, hors'd +Vpon the sightlesse Curriors of the Ayre, +Shall blow the horrid deed in euery eye, +That teares shall drowne the winde. I haue no Spurre +To pricke the sides of my intent, but onely +Vaulting Ambition, which ore-leapes it selfe, +And falles on th' other. +Enter Lady. + +How now? What Newes? + La. He has almost supt: why haue you left the chamber? + Mac. Hath he ask'd for me? + La. Know you not, he ha's? + Mac. We will proceed no further in this Businesse: +He hath Honour'd me of late, and I haue bought +Golden Opinions from all sorts of people, +Which would be worne now in their newest glosse, +Not cast aside so soone + + La. Was the hope drunke, +Wherein you drest your selfe? Hath it slept since? +And wakes it now to looke so greene, and pale, +At what it did so freely? From this time, +Such I account thy loue. Art thou affear'd +To be the same in thine owne Act, and Valour, +As thou art in desire? Would'st thou haue that +Which thou esteem'st the Ornament of Life, +And liue a Coward in thine owne Esteeme? +Letting I dare not, wait vpon I would, +Like the poore Cat i'th' Addage + + Macb. Prythee peace: +I dare do all that may become a man, +Who dares do more, is none + + La. What Beast was't then +That made you breake this enterprize to me? +When you durst do it, then you were a man: +And to be more then what you were, you would +Be so much more the man. Nor time, nor place +Did then adhere, and yet you would make both: +They haue made themselues, and that their fitnesse now +Do's vnmake you. I haue giuen Sucke, and know +How tender 'tis to loue the Babe that milkes me, +I would, while it was smyling in my Face, +Haue pluckt my Nipple from his Bonelesse Gummes, +And dasht the Braines out, had I so sworne +As you haue done to this + + Macb. If we should faile? + Lady. We faile? +But screw your courage to the sticking place, +And wee'le not fayle: when Duncan is asleepe, +(Whereto the rather shall his dayes hard Iourney +Soundly inuite him) his two Chamberlaines +Will I with Wine, and Wassell, so conuince, +That Memorie, the Warder of the Braine, +Shall be a Fume, and the Receit of Reason +A Lymbeck onely: when in Swinish sleepe, +Their drenched Natures lyes as in a Death, +What cannot you and I performe vpon +Th' vnguarded Duncan? What not put vpon +His spungie Officers? who shall beare the guilt +Of our great quell + + Macb. Bring forth Men-Children onely: +For thy vndaunted Mettle should compose +Nothing but Males. Will it not be receiu'd, +When we haue mark'd with blood those sleepie two +Of his owne Chamber, and vs'd their very Daggers, +That they haue don't? + Lady. Who dares receiue it other, +As we shall make our Griefes and Clamor rore, +Vpon his Death? + Macb. I am settled, and bend vp +Each corporall Agent to this terrible Feat. +Away, and mock the time with fairest show, +False Face must hide what the false Heart doth know. + +Exeunt. + + +Actus Secundus. Scena Prima. + +Enter Banquo, and Fleance, with a Torch before him. + + Banq. How goes the Night, Boy? + Fleance. The Moone is downe: I haue not heard the +Clock + + Banq. And she goes downe at Twelue + + Fleance. I take't, 'tis later, Sir + + Banq. Hold, take my Sword: +There's Husbandry in Heauen, +Their Candles are all out: take thee that too. +A heauie Summons lyes like Lead vpon me, +And yet I would not sleepe: +Mercifull Powers, restraine in me the cursed thoughts +That Nature giues way to in repose. +Enter Macbeth, and a Seruant with a Torch. + +Giue me my Sword: who's there? + Macb. A Friend + + Banq. What Sir, not yet at rest? the King's a bed. +He hath beene in vnusuall Pleasure, +And sent forth great Largesse to your Offices. +This Diamond he greetes your Wife withall, +By the name of most kind Hostesse, +And shut vp in measurelesse content + + Mac. Being vnprepar'd, +Our will became the seruant to defect, +Which else should free haue wrought + + Banq. All's well. +I dreamt last Night of the three weyward Sisters: +To you they haue shew'd some truth + + Macb. I thinke not of them: +Yet when we can entreat an houre to serue, +We would spend it in some words vpon that Businesse, +If you would graunt the time + + Banq. At your kind'st leysure + + Macb. If you shall cleaue to my consent, +When 'tis, it shall make Honor for you + + Banq. So I lose none, +In seeking to augment it, but still keepe +My Bosome franchis'd, and Allegeance cleare, +I shall be counsail'd + + Macb. Good repose the while + + Banq. Thankes Sir: the like to you. + +Exit Banquo. + + Macb. Goe bid thy Mistresse, when my drinke is ready, +She strike vpon the Bell. Get thee to bed. +Enter. + +Is this a Dagger, which I see before me, +The Handle toward my Hand? Come, let me clutch thee: +I haue thee not, and yet I see thee still. +Art thou not fatall Vision, sensible +To feeling, as to sight? or art thou but +A Dagger of the Minde, a false Creation, +Proceeding from the heat-oppressed Braine? +I see thee yet, in forme as palpable, +As this which now I draw. +Thou marshall'st me the way that I was going, +And such an Instrument I was to vse. +Mine Eyes are made the fooles o'th' other Sences, +Or else worth all the rest: I see thee still; +And on thy Blade, and Dudgeon, Gouts of Blood, +Which was not so before. There's no such thing: +It is the bloody Businesse, which informes +Thus to mine Eyes. Now o're the one halfe World +Nature seemes dead, and wicked Dreames abuse +The Curtain'd sleepe: Witchcraft celebrates +Pale Heccats Offrings: and wither'd Murther, +Alarum'd by his Centinell, the Wolfe, +Whose howle's his Watch, thus with his stealthy pace, +With Tarquins rauishing sides, towards his designe +Moues like a Ghost. Thou sowre and firme-set Earth +Heare not my steps, which they may walke, for feare +Thy very stones prate of my where-about, +And take the present horror from the time, +Which now sutes with it. Whiles I threat, he liues: +Words to the heat of deedes too cold breath giues. + +A Bell rings. + +I goe, and it is done: the Bell inuites me. +Heare it not, Duncan, for it is a Knell, +That summons thee to Heauen, or to Hell. +Enter. + + +Scena Secunda. + +Enter Lady. + + La. That which hath made the[m] drunk, hath made me bold: +What hath quench'd them, hath giuen me fire. +Hearke, peace: it was the Owle that shriek'd, +The fatall Bell-man, which giues the stern'st good-night. +He is about it, the Doores are open: +And the surfeted Groomes doe mock their charge +With Snores. I haue drugg'd their Possets, +That Death and Nature doe contend about them, +Whether they liue, or dye. +Enter Macbeth. + + Macb. Who's there? what hoa? + Lady. Alack, I am afraid they haue awak'd, +And 'tis not done: th' attempt, and not the deed, +Confounds vs: hearke: I lay'd their Daggers ready, +He could not misse 'em. Had he not resembled +My Father as he slept, I had don't. +My Husband? + Macb. I haue done the deed: +Didst thou not heare a noyse? + Lady. I heard the Owle schreame, and the Crickets cry. +Did not you speake? + Macb. When? + Lady. Now + + Macb. As I descended? + Lady. I + + Macb. Hearke, who lyes i'th' second Chamber? + Lady. Donalbaine + + Mac. This is a sorry sight + + Lady. A foolish thought, to say a sorry sight + + Macb. There's one did laugh in's sleepe, +And one cry'd Murther, that they did wake each other: +I stood, and heard them: But they did say their Prayers, +And addrest them againe to sleepe + + Lady. There are two lodg'd together + + Macb. One cry'd God blesse vs, and Amen the other, +As they had seene me with these Hangmans hands: +Listning their feare, I could not say Amen, +When they did say God blesse vs + + Lady. Consider it not so deepely + + Mac. But wherefore could not I pronounce Amen? +I had most need of Blessing, and Amen stuck in my throat + + Lady. These deeds must not be thought +After these wayes: so, it will make vs mad + + Macb. Me thought I heard a voyce cry, Sleep no more: +Macbeth does murther Sleepe, the innocent Sleepe, +Sleepe that knits vp the rauel'd Sleeue of Care, +The death of each dayes Life, sore Labors Bath, +Balme of hurt Mindes, great Natures second Course, +Chiefe nourisher in Life's Feast + + Lady. What doe you meane? + Macb. Still it cry'd, Sleepe no more to all the House: +Glamis hath murther'd Sleepe, and therefore Cawdor +Shall sleepe no more: Macbeth shall sleepe no more + + Lady. Who was it, that thus cry'd? why worthy Thane, +You doe vnbend your Noble strength, to thinke +So braine-sickly of things: Goe get some Water, +And wash this filthie Witnesse from your Hand. +Why did you bring these Daggers from the place? +They must lye there: goe carry them, and smeare +The sleepie Groomes with blood + + Macb. Ile goe no more: +I am afraid, to thinke what I haue done: +Looke on't againe, I dare not + + Lady. Infirme of purpose: +Giue me the Daggers: the sleeping, and the dead, +Are but as Pictures: 'tis the Eye of Childhood, +That feares a painted Deuill. If he doe bleed, +Ile guild the Faces of the Groomes withall, +For it must seeme their Guilt. +Enter. + +Knocke within. + + Macb. Whence is that knocking? +How is't with me, when euery noyse appalls me? +What Hands are here? hah: they pluck out mine Eyes. +Will all great Neptunes Ocean wash this blood +Cleane from my Hand? no: this my Hand will rather +The multitudinous Seas incarnardine, +Making the Greene one, Red. +Enter Lady. + + Lady. My Hands are of your colour: but I shame +To weare a Heart so white. + +Knocke. + +I heare a knocking at the South entry: +Retyre we to our Chamber: +A little Water cleares vs of this deed. +How easie is it then? your Constancie +Hath left you vnattended. + +Knocke. + +Hearke, more knocking. +Get on your Night-Gowne, least occasion call vs, +And shew vs to be Watchers: be not lost +So poorely in your thoughts + + Macb. To know my deed, + +Knocke. + +'Twere best not know my selfe. +Wake Duncan with thy knocking: +I would thou could'st. + +Exeunt. + + +Scena Tertia. + +Enter a Porter. Knocking within. + + Porter. Here's a knocking indeede: if a man were +Porter of Hell Gate, hee should haue old turning the +Key. + +Knock. + +Knock, Knock, Knock. Who's there +i'th' name of Belzebub? Here's a Farmer, that hang'd +himselfe on th' expectation of Plentie: Come in time, haue +Napkins enow about you, here you'le sweat for't. + +Knock. + +Knock, knock. Who's there in th' other Deuils Name? +Faith here's an Equiuocator, that could sweare in both +the Scales against eyther Scale, who committed Treason +enough for Gods sake, yet could not equiuocate to Heauen: +oh come in, Equiuocator. + +Knock. + +Knock, Knock, Knock. Who's there? 'Faith here's an English +Taylor come hither, for stealing out of a French Hose: +Come in Taylor, here you may rost your Goose. +Knock. + +Knock, Knock. Neuer at quiet: What are you? but this +place is too cold for Hell. Ile Deuill-Porter it no further: +I had thought to haue let in some of all Professions, that +goe the Primrose way to th' euerlasting Bonfire. + +Knock. + +Anon, anon, I pray you remember the Porter. +Enter Macduff, and Lenox. + + Macd. Was it so late, friend, ere you went to Bed, +That you doe lye so late? + Port. Faith Sir, we were carowsing till the second Cock: +And Drinke, Sir, is a great prouoker of three things + + Macd. What three things does Drinke especially +prouoke? + Port. Marry, Sir, Nose-painting, Sleepe, and Vrine. +Lecherie, Sir, it prouokes, and vnprouokes: it prouokes +the desire, but it takes away the performance. Therefore +much Drinke may be said to be an Equiuocator with Lecherie: +it makes him, and it marres him; it sets him on, +and it takes him off; it perswades him, and dis-heartens +him; makes him stand too, and not stand too: in conclusion, +equiuocates him in a sleepe, and giuing him the Lye, +leaues him + + Macd. I beleeue, Drinke gaue thee the Lye last Night + + Port. That it did, Sir, i'the very Throat on me: but I +requited him for his Lye, and (I thinke) being too strong +for him, though he tooke vp my Legges sometime, yet I +made a Shift to cast him. +Enter Macbeth. + + Macd. Is thy Master stirring? +Our knocking ha's awak'd him: here he comes + + Lenox. Good morrow, Noble Sir + + Macb. Good morrow both + + Macd. Is the King stirring, worthy Thane? + Macb. Not yet + + Macd. He did command me to call timely on him, +I haue almost slipt the houre + + Macb. Ile bring you to him + + Macd. I know this is a ioyfull trouble to you: +But yet 'tis one + + Macb. The labour we delight in, Physicks paine: +This is the Doore + + Macd. Ile make so bold to call, for 'tis my limitted +seruice. + +Exit Macduffe. + + Lenox. Goes the King hence to day? + Macb. He does: he did appoint so + + Lenox. The Night ha's been vnruly: +Where we lay, our Chimneys were blowne downe, +And (as they say) lamentings heard i'th' Ayre; +Strange Schreemes of Death, +And Prophecying, with Accents terrible, +Of dyre Combustion, and confus'd Euents, +New hatch'd toth' wofull time. +The obscure Bird clamor'd the liue-long Night. +Some say, the Earth was Feuorous, +And did shake + + Macb. 'Twas a rough Night + + Lenox. My young remembrance cannot paralell +A fellow to it. +Enter Macduff. + + Macd. O horror, horror, horror, +Tongue nor Heart cannot conceiue, nor name thee + + Macb. and Lenox. What's the matter? + Macd. Confusion now hath made his Master-peece: +Most sacrilegious Murther hath broke ope +The Lords anoynted Temple, and stole thence +The Life o'th' Building + + Macb. What is't you say, the Life? + Lenox. Meane you his Maiestie? + Macd. Approch the Chamber, and destroy your sight +With a new Gorgon. Doe not bid me speake: +See, and then speake your selues: awake, awake, + +Exeunt. Macbeth and Lenox. + +Ring the Alarum Bell: Murther, and Treason, +Banquo, and Donalbaine: Malcolme awake, +Shake off this Downey sleepe, Deaths counterfeit, +And looke on Death it selfe: vp, vp, and see +The great Doomes Image: Malcolme, Banquo, +As from your Graues rise vp, and walke like Sprights, +To countenance this horror. Ring the Bell. + +Bell rings. Enter Lady. + + Lady. What's the Businesse? +That such a hideous Trumpet calls to parley +The sleepers of the House? speake, speake + + Macd. O gentle Lady, +'Tis not for you to heare what I can speake: +The repetition in a Womans eare, +Would murther as it fell. +Enter Banquo. + +O Banquo, Banquo, Our Royall Master's murther'd + + Lady. Woe, alas: +What, in our House? + Ban. Too cruell, any where. +Deare Duff, I prythee contradict thy selfe, +And say, it is not so. +Enter Macbeth, Lenox, and Rosse. + + Macb. Had I but dy'd an houre before this chance, +I had liu'd a blessed time: for from this instant, +There's nothing serious in Mortalitie: +All is but Toyes: Renowne and Grace is dead, +The Wine of Life is drawne, and the meere Lees +Is left this Vault, to brag of. +Enter Malcolme and Donalbaine. + + Donal. What is amisse? + Macb. You are, and doe not know't: +The Spring, the Head, the Fountaine of your Blood +Is stopt, the very Source of it is stopt + + Macd. Your Royall Father's murther'd + + Mal. Oh, by whom? + Lenox. Those of his Chamber, as it seem'd, had don't: +Their Hands and Faces were all badg'd with blood, +So were their Daggers, which vnwip'd, we found +Vpon their Pillowes: they star'd, and were distracted, +No mans Life was to be trusted with them + + Macb. O, yet I doe repent me of my furie, +That I did kill them + + Macd. Wherefore did you so? + Macb. Who can be wise, amaz'd, temp'rate, & furious, +Loyall, and Neutrall, in a moment? No man: +Th' expedition of my violent Loue +Out-run the pawser, Reason. Here lay Duncan, +His Siluer skinne, lac'd with His Golden Blood, +And his gash'd Stabs, look'd like a Breach in Nature, +For Ruines wastfull entrance: there the Murtherers, +Steep'd in the Colours of their Trade; their Daggers +Vnmannerly breech'd with gore: who could refraine, +That had a heart to loue; and in that heart, +Courage, to make's loue knowne? + Lady. Helpe me hence, hoa + + Macd. Looke to the Lady + + Mal. Why doe we hold our tongues, +That most may clayme this argument for ours? + Donal. What should be spoken here, +Where our Fate hid in an augure hole, +May rush, and seize vs? Let's away, +Our Teares are not yet brew'd + + Mal. Nor our strong Sorrow +Vpon the foot of Motion + + Banq. Looke to the Lady: +And when we haue our naked Frailties hid, +That suffer in exposure; let vs meet, +And question this most bloody piece of worke, +To know it further. Feares and scruples shake vs: +In the great Hand of God I stand, and thence, +Against the vndivulg'd pretence, I fight +Of Treasonous Mallice + + Macd. And so doe I + + All. So all + + Macb. Let's briefely put on manly readinesse, +And meet i'th' Hall together + + All. Well contented. + +Exeunt. + + Malc. What will you doe? +Let's not consort with them: +To shew an vnfelt Sorrow, is an Office +Which the false man do's easie. +Ile to England + + Don. To Ireland, I: +Our seperated fortune shall keepe vs both the safer: +Where we are, there's Daggers in mens smiles; +The neere in blood, the neerer bloody + + Malc. This murtherous Shaft that's shot, +Hath not yet lighted: and our safest way, +Is to auoid the ayme. Therefore to Horse, +And let vs not be daintie of leaue-taking, +But shift away: there's warrant in that Theft, +Which steales it selfe, when there's no mercie left. + +Exeunt. + + + +Scena Quarta. + +Enter Rosse, with an Old man. + + Old man. Threescore and ten I can remember well, +Within the Volume of which Time, I haue seene +Houres dreadfull, and things strange: but this sore Night +Hath trifled former knowings + + Rosse. Ha, good Father, +Thou seest the Heauens, as troubled with mans Act, +Threatens his bloody Stage: byth' Clock 'tis Day, +And yet darke Night strangles the trauailing Lampe: +Is't Nights predominance, or the Dayes shame, +That Darknesse does the face of Earth intombe, +When liuing Light should kisse it? + Old man. 'Tis vnnaturall, +Euen like the deed that's done: On Tuesday last, +A Faulcon towring in her pride of place, +Was by a Mowsing Owle hawkt at, and kill'd + + Rosse. And Duncans Horses, +(A thing most strange, and certaine) +Beauteous, and swift, the Minions of their Race, +Turn'd wilde in nature, broke their stalls, flong out, +Contending 'gainst Obedience, as they would +Make Warre with Mankinde + + Old man. 'Tis said, they eate each other + + Rosse. They did so: +To th' amazement of mine eyes that look'd vpon't. +Enter Macduffe. + +Heere comes the good Macduffe. +How goes the world Sir, now? + Macd. Why see you not? + Ross. Is't known who did this more then bloody deed? + Macd. Those that Macbeth hath slaine + + Ross. Alas the day, +What good could they pretend? + Macd. They were subborned, +Malcolme, and Donalbaine the Kings two Sonnes +Are stolne away and fled, which puts vpon them +Suspition of the deed + + Rosse. 'Gainst Nature still, +Thriftlesse Ambition, that will rauen vp +Thine owne liues meanes: Then 'tis most like, +The Soueraignty will fall vpon Macbeth + + Macd. He is already nam'd, and gone to Scone +To be inuested + + Rosse. Where is Duncans body? + Macd. Carried to Colmekill, +The Sacred Store-house of his Predecessors, +And Guardian of their Bones + + Rosse. Will you to Scone? + Macd. No Cosin, Ile to Fife + + Rosse. Well, I will thither + + Macd. Well may you see things wel done there: Adieu +Least our old Robes sit easier then our new + + Rosse. Farewell, Father + + Old M. Gods benyson go with you, and with those +That would make good of bad, and Friends of Foes. + +Exeunt. omnes + +Actus Tertius. Scena Prima. + +Enter Banquo. + + Banq. Thou hast it now, King, Cawdor, Glamis, all, +As the weyard Women promis'd, and I feare +Thou playd'st most fowly for't: yet it was saide +It should not stand in thy Posterity, +But that my selfe should be the Roote, and Father +Of many Kings. If there come truth from them, +As vpon thee Macbeth, their Speeches shine, +Why by the verities on thee made good, +May they not be my Oracles as well, +And set me vp in hope. But hush, no more. + +Senit sounded. Enter Macbeth as King, Lady Lenox, Rosse, Lords, +and +Attendants. + + Macb. Heere's our chiefe Guest + + La. If he had beene forgotten, +It had bene as a gap in our great Feast, +And all-thing vnbecomming + + Macb. To night we hold a solemne Supper sir, +And Ile request your presence + + Banq. Let your Highnesse +Command vpon me, to the which my duties +Are with a most indissoluble tye +For euer knit + + Macb. Ride you this afternoone? + Ban. I, my good Lord + + Macb. We should haue else desir'd your good aduice +(Which still hath been both graue, and prosperous) +In this dayes Councell: but wee'le take to morrow. +Is't farre you ride? + Ban. As farre, my Lord, as will fill vp the time +'Twixt this, and Supper. Goe not my Horse the better, +I must become a borrower of the Night, +For a darke houre, or twaine + + Macb. Faile not our Feast + + Ban. My Lord, I will not + + Macb. We heare our bloody Cozens are bestow'd +In England, and in Ireland, not confessing +Their cruell Parricide, filling their hearers +With strange inuention. But of that to morrow, +When therewithall, we shall haue cause of State, +Crauing vs ioyntly. Hye you to Horse: +Adieu, till you returne at Night. +Goes Fleance with you? + Ban. I, my good Lord: our time does call vpon's + + Macb. I wish your Horses swift, and sure of foot: +And so I doe commend you to their backs. +Farwell. + +Exit Banquo. + +Let euery man be master of his time, +Till seuen at Night, to make societie +The sweeter welcome: +We will keepe our selfe till Supper time alone: +While then, God be with you. + +Exeunt. Lords. + +Sirrha, a word with you: Attend those men +Our pleasure? + Seruant. They are, my Lord, without the Pallace +Gate + + Macb. Bring them before vs. + +Exit Seruant. + +To be thus, is nothing, but to be safely thus +Our feares in Banquo sticke deepe, +And in his Royaltie of Nature reignes that +Which would be fear'd. 'Tis much he dares, +And to that dauntlesse temper of his Minde, +He hath a Wisdome, that doth guide his Valour, +To act in safetie. There is none but he, +Whose being I doe feare: and vnder him, +My Genius is rebuk'd, as it is said +Mark Anthonies was by Caesar. He chid the Sisters, +When first they put the Name of King vpon me, +And bad them speake to him. Then Prophet-like, +They hayl'd him Father to a Line of Kings. +Vpon my Head they plac'd a fruitlesse Crowne, +And put a barren Scepter in my Gripe, +Thence to be wrencht with an vnlineall Hand, +No Sonne of mine succeeding: if't be so, +For Banquo's Issue haue I fil'd my Minde, +For them, the gracious Duncan haue I murther'd, +Put Rancours in the Vessell of my Peace +Onely for them, and mine eternall Iewell +Giuen to the common Enemie of Man, +To make them Kings, the Seedes of Banquo Kings. +Rather then so, come Fate into the Lyst, +And champion me to th' vtterance. +Who's there? +Enter Seruant, and two Murtherers. + +Now goe to the Doore, and stay there till we call. + +Exit Seruant. + +Was it not yesterday we spoke together? + Murth. It was, so please your Highnesse + + Macb. Well then, +Now haue you consider'd of my speeches: +Know, that it was he, in the times past, +Which held you so vnder fortune, +Which you thought had been our innocent selfe. +This I made good to you, in our last conference, +Past in probation with you: +How you were borne in hand, how crost: +The Instruments: who wrought with them: +And all things else, that might +To halfe a Soule, and to a Notion craz'd, +Say, Thus did Banquo + + 1.Murth. You made it knowne to vs + + Macb. I did so: +And went further, which is now +Our point of second meeting. +Doe you finde your patience so predominant, +In your nature, that you can let this goe? +Are you so Gospell'd, to pray for this good man, +And for his Issue, whose heauie hand +Hath bow'd you to the Graue, and begger'd +Yours for euer? + 1.Murth. We are men, my Liege + + Macb. I, in the Catalogue ye goe for men, +As Hounds, and Greyhounds, Mungrels, Spaniels, Curres, +Showghes, Water-Rugs, and Demy-Wolues are clipt +All by the Name of Dogges: the valued file +Distinguishes the swift, the slow, the subtle, +The House-keeper, the Hunter, euery one +According to the gift, which bounteous Nature +Hath in him clos'd: whereby he does receiue +Particular addition, from the Bill, +That writes them all alike: and so of men. +Now, if you haue a station in the file, +Not i'th' worst ranke of Manhood, say't, +And I will put that Businesse in your Bosomes, +Whose execution takes your Enemie off, +Grapples you to the heart; and loue of vs, +Who weare our Health but sickly in his Life, +Which in his Death were perfect + + 2.Murth. I am one, my Liege, +Whom the vile Blowes and Buffets of the World +Hath so incens'd, that I am recklesse what I doe, +To spight the World + + 1.Murth. And I another, +So wearie with Disasters, tugg'd with Fortune, +That I would set my Life on any Chance, +To mend it, or be rid on't + + Macb. Both of you know Banquo was your Enemie + + Murth. True, my Lord + + Macb. So is he mine: and in such bloody distance, +That euery minute of his being, thrusts +Against my neer'st of Life: and though I could +With bare-fac'd power sweepe him from my sight, +And bid my will auouch it; yet I must not, +For certaine friends that are both his, and mine, +Whose loues I may not drop, but wayle his fall, +Who I my selfe struck downe: and thence it is, +That I to your assistance doe make loue, +Masking the Businesse from the common Eye, +For sundry weightie Reasons + + 2.Murth. We shall, my Lord, +Performe what you command vs + + 1.Murth. Though our Liues- + Macb. Your Spirits shine through you. +Within this houre, at most, +I will aduise you where to plant your selues, +Acquaint you with the perfect Spy o'th' time, +The moment on't, for't must be done to Night, +And something from the Pallace: alwayes thought, +That I require a clearenesse; and with him, +To leaue no Rubs nor Botches in the Worke: + Fleans , his Sonne, that keepes him companie, +Whose absence is no lesse materiall to me, +Then is his Fathers, must embrace the fate +Of that darke houre: resolue your selues apart, +Ile come to you anon + + Murth. We are resolu'd, my Lord + + Macb. Ile call vpon you straight: abide within, +It is concluded: Banquo, thy Soules flight, +If it finde Heauen, must finde it out to Night. + +Exeunt. + + +Scena Secunda. + +Enter Macbeths Lady, and a Seruant. + + Lady. Is Banquo gone from Court? + Seruant. I, Madame, but returnes againe to Night + + Lady. Say to the King, I would attend his leysure, +For a few words + + Seruant. Madame, I will. +Enter. + + Lady. Nought's had, all's spent. +Where our desire is got without content: +'Tis safer, to be that which we destroy, +Then by destruction dwell in doubtfull ioy. +Enter Macbeth. + +How now, my Lord, why doe you keepe alone? +Of sorryest Fancies your Companions making, +Vsing those Thoughts, which should indeed haue dy'd +With them they thinke on: things without all remedie +Should be without regard: what's done, is done + + Macb. We haue scorch'd the Snake, not kill'd it: +Shee'le close, and be her selfe, whilest our poore Mallice +Remaines in danger of her former Tooth. +But let the frame of things dis-ioynt, +Both the Worlds suffer, +Ere we will eate our Meale in feare, and sleepe +In the affliction of these terrible Dreames, +That shake vs Nightly: Better be with the dead, +Whom we, to gayne our peace, haue sent to peace, +Then on the torture of the Minde to lye +In restlesse extasie. +Duncane is in his Graue: +After Lifes fitfull Feuer, he sleepes well, +Treason ha's done his worst: nor Steele, nor Poyson, +Mallice domestique, forraine Leuie, nothing, +Can touch him further + + Lady. Come on: +Gentle my Lord, sleeke o're your rugged Lookes, +Be bright and Iouiall among your Guests to Night + + Macb. So shall I Loue, and so I pray be you: +Let your remembrance apply to Banquo, +Present him Eminence, both with Eye and Tongue: +Vnsafe the while, that wee must laue +Our Honors in these flattering streames, +And make our Faces Vizards to our Hearts, +Disguising what they are + + Lady. You must leaue this + + Macb. O, full of Scorpions is my Minde, deare Wife: +Thou know'st, that Banquo and his Fleans liues + + Lady. But in them, Natures Coppie's not eterne + + Macb. There's comfort yet, they are assaileable, +Then be thou iocund: ere the Bat hath flowne +His Cloyster'd flight, ere to black Heccats summons +The shard-borne Beetle, with his drowsie hums, +Hath rung Nights yawning Peale, +There shall be done a deed of dreadfull note + + Lady. What's to be done? + Macb. Be innocent of the knowledge, dearest Chuck, +Till thou applaud the deed: Come, seeling Night, +Skarfe vp the tender Eye of pittifull Day, +And with thy bloodie and inuisible Hand +Cancell and teare to pieces that great Bond, +Which keepes me pale. Light thickens, +And the Crow makes Wing toth' Rookie Wood: +Good things of Day begin to droope, and drowse, +Whiles Nights black Agents to their Prey's doe rowse. +Thou maruell'st at my words: but hold thee still, +Things bad begun, make strong themselues by ill: +So prythee goe with me. + +Exeunt. + + +Scena Tertia. + +Enter three Murtherers. + + 1. But who did bid thee ioyne with vs? + 3. Macbeth + + 2. He needes not our mistrust, since he deliuers +Our Offices, and what we haue to doe, +To the direction iust + + 1. Then stand with vs: +The West yet glimmers with some streakes of Day. +Now spurres the lated Traueller apace, +To gayne the timely Inne, and neere approches +The subiect of our Watch + + 3. Hearke, I heare Horses + + Banquo within. Giue vs a Light there, hoa + + 2. Then 'tis hee: +The rest, that are within the note of expectation, +Alreadie are i'th' Court + + 1. His Horses goe about + + 3. Almost a mile: but he does vsually, +So all men doe, from hence toth' Pallace Gate +Make it their Walke. +Enter Banquo and Fleans, with a Torch. + + 2. A Light, a Light + + 3. 'Tis hee + + 1. Stand too't + + Ban. It will be Rayne to Night + + 1. Let it come downe + + Ban. O, Trecherie! +Flye good Fleans, flye, flye, flye, +Thou may'st reuenge. O Slaue! + 3. Who did strike out the Light? + 1. Was't not the way? + 3. There's but one downe: the Sonne is fled + + 2. We haue lost +Best halfe of our Affaire + + 1. Well, let's away, and say how much is done. + +Exeunt. + + +Scaena Quarta. + +Banquet prepar'd. Enter Macbeth, Lady, Rosse, Lenox, Lords, and +Attendants. + + Macb. You know your owne degrees, sit downe: +At first and last, the hearty welcome + + Lords. Thankes to your Maiesty + + Macb. Our selfe will mingle with Society, +And play the humble Host: +Our Hostesse keepes her State, but in best time +We will require her welcome + + La. Pronounce it for me Sir, to all our Friends, +For my heart speakes, they are welcome. +Enter first Murtherer. + + Macb. See they encounter thee with their harts thanks +Both sides are euen: heere Ile sit i'th' mid'st, +Be large in mirth, anon wee'l drinke a Measure +The Table round. There's blood vpon thy face + + Mur. 'Tis Banquo's then + + Macb. 'Tis better thee without, then he within. +Is he dispatch'd? + Mur. My Lord his throat is cut, that I did for him + + Mac. Thou art the best o'th' Cut-throats, +Yet hee's good that did the like for Fleans: +If thou did'st it, thou art the Non-pareill + + Mur. Most Royall Sir +Fleans is scap'd + + Macb. Then comes my Fit againe: +I had else beene perfect; +Whole as the Marble, founded as the Rocke, +As broad, and generall, as the casing Ayre: +But now I am cabin'd, crib'd, confin'd, bound in +To sawcy doubts, and feares. But Banquo's safe? + Mur. I, my good Lord: safe in a ditch he bides, +With twenty trenched gashes on his head; +The least a Death to Nature + + Macb. Thankes for that: +There the growne Serpent lyes, the worme that's fled +Hath Nature that in time will Venom breed, +No teeth for th' present. Get thee gone, to morrow +Wee'l heare our selues againe. + +Exit Murderer. + + Lady. My Royall Lord, +You do not giue the Cheere, the Feast is sold +That is not often vouch'd, while 'tis a making: +'Tis giuen, with welcome: to feede were best at home: +From thence, the sawce to meate is Ceremony, +Meeting were bare without it. +Enter the Ghost of Banquo, and sits in Macbeths place. + + Macb. Sweet Remembrancer: +Now good digestion waite on Appetite, +And health on both + + Lenox. May't please your Highnesse sit + + Macb. Here had we now our Countries Honor, roof'd, +Were the grac'd person of our Banquo present: +Who, may I rather challenge for vnkindnesse, +Then pitty for Mischance + + Rosse. His absence (Sir) +Layes blame vpon his promise. Pleas't your Highnesse +To grace vs with your Royall Company? + Macb. The Table's full + + Lenox. Heere is a place reseru'd Sir + + Macb. Where? + Lenox. Heere my good Lord. +What is't that moues your Highnesse? + Macb. Which of you haue done this? + Lords. What, my good Lord? + Macb. Thou canst not say I did it: neuer shake +Thy goary lockes at me + + Rosse. Gentlemen rise, his Highnesse is not well + + Lady. Sit worthy Friends: my Lord is often thus, +And hath beene from his youth. Pray you keepe Seat, +The fit is momentary, vpon a thought +He will againe be well. If much you note him +You shall offend him, and extend his Passion, +Feed, and regard him not. Are you a man? + Macb. I, and a bold one, that dare looke on that +Which might appall the Diuell + + La. O proper stuffe: +This is the very painting of your feare: +This is the Ayre-drawne-Dagger which you said +Led you to Duncan. O, these flawes and starts +(Impostors to true feare) would well become +A womans story, at a Winters fire +Authoriz'd by her Grandam: shame it selfe, +Why do you make such faces? When all's done +You looke but on a stoole + + Macb. Prythee see there: +Behold, looke, loe, how say you: +Why what care I, if thou canst nod, speake too. +If Charnell houses, and our Graues must send +Those that we bury, backe; our Monuments +Shall be the Mawes of Kytes + + La. What? quite vnmann'd in folly + + Macb. If I stand heere, I saw him + + La. Fie for shame + + Macb. Blood hath bene shed ere now, i'th' olden time +Ere humane Statute purg'd the gentle Weale: +I, and since too, Murthers haue bene perform'd +Too terrible for the eare. The times has bene, +That when the Braines were out, the man would dye, +And there an end: But now they rise againe +With twenty mortall murthers on their crownes, +And push vs from our stooles. This is more strange +Then such a murther is + + La. My worthy Lord +Your Noble Friends do lacke you + + Macb. I do forget: +Do not muse at me my most worthy Friends, +I haue a strange infirmity, which is nothing +To those that know me. Come, loue and health to all, +Then Ile sit downe: Giue me some Wine, fill full: +Enter Ghost. + +I drinke to th' generall ioy o'th' whole Table, +And to our deere Friend Banquo, whom we misse: +Would he were heere: to all, and him we thirst, +And all to all + + Lords. Our duties, and the pledge + + Mac. Auant, & quit my sight, let the earth hide thee: +Thy bones are marrowlesse, thy blood is cold: +Thou hast no speculation in those eyes +Which thou dost glare with + + La. Thinke of this good Peeres +But as a thing of Custome: 'Tis no other, +Onely it spoyles the pleasure of the time + + Macb. What man dare, I dare: +Approach thou like the rugged Russian Beare, +The arm'd Rhinoceros, or th' Hircan Tiger, +Take any shape but that, and my firme Nerues +Shall neuer tremble. Or be aliue againe, +And dare me to the Desart with thy Sword: +If trembling I inhabit then, protest mee +The Baby of a Girle. Hence horrible shadow, +Vnreall mock'ry hence. Why so, being gone +I am a man againe: pray you sit still + + La. You haue displac'd the mirth, +Broke the good meeting, with most admir'd disorder + + Macb. Can such things be, +And ouercome vs like a Summers Clowd, +Without our speciall wonder? You make me strange +Euen to the disposition that I owe, +When now I thinke you can behold such sights, +And keepe the naturall Rubie of your Cheekes, +When mine is blanch'd with feare + + Rosse. What sights, my Lord? + La. I pray you speake not: he growes worse & worse +Question enrages him: at once, goodnight. +Stand not vpon the order of your going, +But go at once + + Len. Good night, and better health +Attend his Maiesty + + La. A kinde goodnight to all. + +Exit Lords. + + Macb. It will haue blood they say: +Blood will haue Blood: +Stones haue beene knowne to moue, & Trees to speake: +Augures, and vnderstood Relations, haue +By Maggot Pyes, & Choughes, & Rookes brought forth +The secret'st man of Blood. What is the night? + La. Almost at oddes with morning, which is which + + Macb. How say'st thou that Macduff denies his person +At our great bidding + + La. Did you send to him Sir? + Macb. I heare it by the way: But I will send: +There's not a one of them but in his house +I keepe a Seruant Feed. I will to morrow +(And betimes I will) to the weyard Sisters. +More shall they speake: for now I am bent to know +By the worst meanes, the worst, for mine owne good, +All causes shall giue way. I am in blood +Stept in so farre, that should I wade no more, +Returning were as tedious as go ore: +Strange things I haue in head, that will to hand, +Which must be acted, ere they may be scand + + La. You lacke the season of all Natures, sleepe + + Macb. Come, wee'l to sleepe: My strange & self-abuse +Is the initiate feare, that wants hard vse: +We are yet but yong indeed. + +Exeunt. + + +Scena Quinta. + +Thunder. Enter the three Witches, meeting Hecat. + + 1. Why how now Hecat, you looke angerly? + Hec. Haue I not reason (Beldams) as you are? +Sawcy, and ouer-bold, how did you dare +To Trade, and Trafficke with Macbeth, +In Riddles, and Affaires of death; +And I the Mistris of your Charmes, +The close contriuer of all harmes, +Was neuer call'd to beare my part, +Or shew the glory of our Art? +And which is worse, all you haue done +Hath bene but for a wayward Sonne, +Spightfull, and wrathfull, who (as others do) +Loues for his owne ends, not for you. +But make amends now: Get you gon, +And at the pit of Acheron +Meete me i'th' Morning: thither he +Will come, to know his Destinie. +Your Vessels, and your Spels prouide, +Your Charmes, and euery thing beside; +I am for th' Ayre: This night Ile spend +Vnto a dismall, and a Fatall end. +Great businesse must be wrought ere Noone. +Vpon the Corner of the Moone +There hangs a vap'rous drop, profound, +Ile catch it ere it come to ground; +And that distill'd by Magicke slights, +Shall raise such Artificiall Sprights, +As by the strength of their illusion, +Shall draw him on to his Confusion. +He shall spurne Fate, scorne Death, and beare +His hopes 'boue Wisedome, Grace, and Feare: +And you all know, Security +Is Mortals cheefest Enemie. + +Musicke, and a Song. + +Hearke, I am call'd: my little Spirit see +Sits in Foggy cloud, and stayes for me. + +Sing within. Come away, come away, &c. + + 1 Come, let's make hast, shee'l soone be +Backe againe. + +Exeunt. + + +Scaena Sexta. + +Enter Lenox, and another Lord. + + Lenox. My former Speeches, +Haue but hit your Thoughts +Which can interpret farther: Onely I say +Things haue bin strangely borne. The gracious Duncan +Was pittied of Macbeth: marry he was dead: +And the right valiant Banquo walk'd too late, +Whom you may say (if't please you) Fleans kill'd, +For Fleans fled: Men must not walke too late. +Who cannot want the thought, how monstrous +It was for Malcolme, and for Donalbane +To kill their gracious Father? Damned Fact, +How it did greeue Macbeth? Did he not straight +In pious rage, the two delinquents teare, +That were the Slaues of drinke, and thralles of sleepe? +Was not that Nobly done? I, and wisely too: +For 'twould haue anger'd any heart aliue +To heare the men deny't. So that I say, +He ha's borne all things well, and I do thinke, +That had he Duncans Sonnes vnder his Key, +(As, and't please Heauen he shall not) they should finde +What 'twere to kill a Father: So should Fleans. +But peace; for from broad words, and cause he fayl'd +His presence at the Tyrants Feast, I heare +Macduffe liues in disgrace. Sir, can you tell +Where he bestowes himselfe? + Lord. The Sonnes of Duncane +(From whom this Tyrant holds the due of Birth) +Liues in the English Court, and is receyu'd +Of the most Pious Edward, with such grace, +That the maleuolence of Fortune, nothing +Takes from his high respect. Thither Macduffe +Is gone, to pray the Holy King, vpon his ayd +To wake Northumberland, and warlike Seyward, +That by the helpe of these (with him aboue) +To ratifie the Worke) we may againe +Giue to our Tables meate, sleepe to our Nights: +Free from our Feasts, and Banquets bloody kniues; +Do faithfull Homage, and receiue free Honors, +All which we pine for now. And this report +Hath so exasperate their King, that hee +Prepares for some attempt of Warre + + Len. Sent he to Macduffe? + Lord. He did: and with an absolute Sir, not I +The clowdy Messenger turnes me his backe, +And hums; as who should say, you'l rue the time +That clogges me with this Answer + + Lenox. And that well might +Aduise him to a Caution, t' hold what distance +His wisedome can prouide. Some holy Angell +Flye to the Court of England, and vnfold +His Message ere he come, that a swift blessing +May soone returne to this our suffering Country, +Vnder a hand accurs'd + + Lord. Ile send my Prayers with him. + +Exeunt. + +Actus Quartus. Scena Prima. + +Thunder. Enter the three Witches. + + 1 Thrice the brinded Cat hath mew'd + + 2 Thrice, and once the Hedge-Pigge whin'd + + 3 Harpier cries, 'tis time, 'tis time + + 1 Round about the Caldron go: +In the poysond Entrailes throw +Toad, that vnder cold stone, +Dayes and Nights, ha's thirty one: +Sweltred Venom sleeping got, +Boyle thou first i'th' charmed pot + + All. Double, double, toile and trouble; +Fire burne, and Cauldron bubble + + 2 Fillet of a Fenny Snake, +In the Cauldron boyle and bake: +Eye of Newt, and Toe of Frogge, +Wooll of Bat, and Tongue of Dogge: +Adders Forke, and Blinde-wormes Sting, +Lizards legge, and Howlets wing: +For a Charme of powrefull trouble, +Like a Hell-broth, boyle and bubble + + All. Double, double, toyle and trouble, +Fire burne, and Cauldron bubble + + 3 Scale of Dragon, Tooth of Wolfe, +Witches Mummey, Maw, and Gulfe +Of the rauin'd salt Sea sharke: +Roote of Hemlocke, digg'd i'th' darke: +Liuer of Blaspheming Iew, +Gall of Goate, and Slippes of Yew, +Sliuer'd in the Moones Ecclipse: +Nose of Turke, and Tartars lips: +Finger of Birth-strangled Babe, +Ditch-deliuer'd by a Drab, +Make the Grewell thicke, and slab. +Adde thereto a Tigers Chawdron, +For th' Ingredience of our Cawdron + + All. Double, double, toyle and trouble, +Fire burne, and Cauldron bubble + + 2 Coole it with a Baboones blood, +Then the Charme is firme and good. +Enter Hecat, and the other three Witches. + + Hec. O well done: I commend your paines, +And euery one shall share i'th' gaines: +And now about the Cauldron sing +Like Elues and Fairies in a Ring, +Inchanting all that you put in. + +Musicke and a Song. Blacke Spirits, &c. + + 2 By the pricking of my Thumbes, +Something wicked this way comes: +Open Lockes, who euer knockes. +Enter Macbeth. + + Macb. How now you secret, black, & midnight Hags? +What is't you do? + All. A deed without a name + + Macb. I coniure you, by that which you Professe, +(How ere you come to know it) answer me: +Though you vntye the Windes, and let them fight +Against the Churches: Though the yesty Waues +Confound and swallow Nauigation vp: +Though bladed Corne be lodg'd, & Trees blown downe, +Though Castles topple on their Warders heads: +Though Pallaces, and Pyramids do slope +Their heads to their Foundations: Though the treasure +Of Natures Germaine, tumble altogether, +Euen till destruction sicken: Answer me +To what I aske you + + 1 Speake + + 2 Demand + + 3 Wee'l answer + + 1 Say, if th'hadst rather heare it from our mouthes, +Or from our Masters + + Macb. Call 'em: let me see 'em + + 1 Powre in Sowes blood, that hath eaten +Her nine Farrow: Greaze that's sweaten +From the Murderers Gibbet, throw +Into the Flame + + All. Come high or low: +Thy Selfe and Office deaftly show. +Thunder. 1. Apparation, an Armed Head. + + Macb. Tell me, thou vnknowne power + + 1 He knowes thy thought: +Heare his speech, but say thou nought + + 1 Appar. Macbeth, Macbeth, Macbeth: +Beware Macduffe, +Beware the Thane of Fife: dismisse me. Enough. + +He Descends. + + Macb. What ere thou art, for thy good caution, thanks +Thou hast harp'd my feare aright. But one word more + + 1 He will not be commanded: heere's another +More potent then the first. + +Thunder. 2 Apparition, a Bloody Childe. + + 2 Appar. Macbeth, Macbeth, Macbeth + + Macb. Had I three eares, Il'd heare thee + + Appar. Be bloody, bold, & resolute: +Laugh to scorne +The powre of man: For none of woman borne +Shall harme Macbeth. + +Descends. + + Mac. Then liue Macduffe: what need I feare of thee? +But yet Ile make assurance: double sure, +And take a Bond of Fate: thou shalt not liue, +That I may tell pale-hearted Feare, it lies; +And sleepe in spight of Thunder. + +Thunder 3 Apparation, a Childe Crowned, with a Tree in his hand. + +What is this, that rises like the issue of a King, +And weares vpon his Baby-brow, the round +And top of Soueraignty? + All. Listen, but speake not too't + + 3 Appar. Be Lyon metled, proud, and take no care: +Who chafes, who frets, or where Conspirers are: +Macbeth shall neuer vanquish'd be, vntill +Great Byrnam Wood, to high Dunsmane Hill +Shall come against him. + +Descend. + + Macb. That will neuer bee: +Who can impresse the Forrest, bid the Tree +Vnfixe his earth-bound Root? Sweet boadments, good: +Rebellious dead, rise neuer till the Wood +Of Byrnan rise, and our high plac'd Macbeth +Shall liue the Lease of Nature, pay his breath +To time, and mortall Custome. Yet my Hart +Throbs to know one thing: Tell me, if your Art +Can tell so much: Shall Banquo's issue euer +Reigne in this Kingdome? + All. Seeke to know no more + + Macb. I will be satisfied. Deny me this, +And an eternall Curse fall on you: Let me know. +Why sinkes that Caldron? & what noise is this? + +Hoboyes + + 1 Shew + + 2 Shew + + 3 Shew + + All. Shew his Eyes, and greeue his Hart, +Come like shadowes, so depart. + +A shew of eight Kings, and Banquo last, with a glasse in his hand. + + Macb. Thou art too like the Spirit of Banquo: Down: +Thy Crowne do's seare mine Eye-bals. And thy haire +Thou other Gold-bound-brow, is like the first: +A third, is like the former. Filthy Hagges, +Why do you shew me this? - A fourth? Start eyes! +What will the Line stretch out to'th' cracke of Doome? +Another yet? A seauenth? Ile see no more: +And yet the eighth appeares, who beares a glasse, +Which shewes me many more: and some I see, +That two-fold Balles, and trebble Scepters carry. +Horrible sight: Now I see 'tis true, +For the Blood-bolter'd Banquo smiles vpon me, +And points at them for his. What? is this so? + 1 I Sir, all this is so. But why +Stands Macbeth thus amazedly? +Come Sisters, cheere we vp his sprights, +And shew the best of our delights. +Ile Charme the Ayre to giue a sound, +While you performe your Antique round: +That this great King may kindly say, +Our duties, did his welcome pay. + +Musicke. The Witches Dance, and vanish. + + Macb. Where are they? Gone? +Let this pernitious houre, +Stand aye accursed in the Kalender. +Come in, without there. +Enter Lenox. + + Lenox. What's your Graces will + + Macb. Saw you the Weyard Sisters? + Lenox. No my Lord + + Macb. Came they not by you? + Lenox. No indeed my Lord + + Macb. Infected be the Ayre whereon they ride, +And damn'd all those that trust them. I did heare +The gallopping of Horse. Who was't came by? + Len. 'Tis two or three my Lord, that bring you word: +Macduff is fled to England + + Macb. Fled to England? + Len. I, my good Lord + + Macb. Time, thou anticipat'st my dread exploits: +The flighty purpose neuer is o're-tooke +Vnlesse the deed go with it. From this moment, +The very firstlings of my heart shall be +The firstlings of my hand. And euen now +To Crown my thoughts with Acts: be it thoght & done: +The Castle of Macduff, I will surprize. +Seize vpon Fife; giue to th' edge o'th' Sword +His Wife, his Babes, and all vnfortunate Soules +That trace him in his Line. No boasting like a Foole, +This deed Ile do, before this purpose coole, +But no more sights. Where are these Gentlemen? +Come bring me where they are. + +Exeunt. + +Scena Secunda. + +Enter Macduffes Wife, her Son, and Rosse. + + Wife. What had he done, to make him fly the Land? + Rosse. You must haue patience Madam + + Wife. He had none: +His flight was madnesse: when our Actions do not, +Our feares do make vs Traitors + + Rosse. You know not +Whether it was his wisedome, or his feare + + Wife. Wisedom? to leaue his wife, to leaue his Babes, +His Mansion, and his Titles, in a place +From whence himselfe do's flye? He loues vs not, +He wants the naturall touch. For the poore Wren +(The most diminitiue of Birds) will fight, +Her yong ones in her Nest, against the Owle: +All is the Feare, and nothing is the Loue; +As little is the Wisedome, where the flight +So runnes against all reason + + Rosse. My deerest Cooz, +I pray you schoole your selfe. But for your Husband, +He is Noble, Wise, Iudicious, and best knowes +The fits o'th' Season. I dare not speake much further, +But cruell are the times, when we are Traitors +And do not know our selues: when we hold Rumor +From what we feare, yet know not what we feare, +But floate vpon a wilde and violent Sea +Each way, and moue. I take my leaue of you: +Shall not be long but Ile be heere againe: +Things at the worst will cease, or else climbe vpward, +To what they were before. My pretty Cosine, +Blessing vpon you + + Wife. Father'd he is, +And yet hee's Father-lesse + + Rosse. I am so much a Foole, should I stay longer +It would be my disgrace, and your discomfort. +I take my leaue at once. + +Exit Rosse. + + Wife. Sirra, your Fathers dead, +And what will you do now? How will you liue? + Son. As Birds do Mother + + Wife. What with Wormes, and Flyes? + Son. With what I get I meane, and so do they + + Wife. Poore Bird, +Thou'dst neuer Feare the Net, nor Lime, +The Pitfall, nor the Gin + + Son. Why should I Mother? +Poore Birds they are not set for: +My Father is not dead for all your saying + + Wife. Yes, he is dead: +How wilt thou do for a Father? + Son. Nay how will you do for a Husband? + Wife. Why I can buy me twenty at any Market + + Son. Then you'l by 'em to sell againe + + Wife. Thou speak'st withall thy wit, +And yet I'faith with wit enough for thee + + Son. Was my Father a Traitor, Mother? + Wife. I, that he was + + Son. What is a Traitor? + Wife. Why one that sweares, and lyes + + Son. And be all Traitors, that do so + + Wife. Euery one that do's so, is a Traitor, +And must be hang'd + + Son. And must they all be hang'd, that swear and lye? + Wife. Euery one + + Son. Who must hang them? + Wife. Why, the honest men + + Son. Then the Liars and Swearers are Fools: for there +are Lyars and Swearers enow, to beate the honest men, +and hang vp them + + Wife. Now God helpe thee, poore Monkie: +But how wilt thou do for a Father? + Son. If he were dead, youl'd weepe for him: if you +would not, it were a good signe, that I should quickely +haue a new Father + + Wife. Poore pratler, how thou talk'st? +Enter a Messenger. + + Mes. Blesse you faire Dame: I am not to you known, +Though in your state of Honor I am perfect; +I doubt some danger do's approach you neerely. +If you will take a homely mans aduice, +Be not found heere: Hence with your little ones +To fright you thus. Me thinkes I am too sauage: +To do worse to you, were fell Cruelty, +Which is too nie your person. Heauen preserue you, +I dare abide no longer. + +Exit Messenger + + Wife. Whether should I flye? +I haue done no harme. But I remember now +I am in this earthly world: where to do harme +Is often laudable, to do good sometime +Accounted dangerous folly. Why then (alas) +Do I put vp that womanly defence, +To say I haue done no harme? +What are these faces? +Enter Murtherers. + + Mur. Where is your Husband? + Wife. I hope in no place so vnsanctified, +Where such as thou may'st finde him + + Mur. He's a Traitor + + Son. Thou ly'st thou shagge-ear'd Villaine + + Mur. What you Egge? +Yong fry of Treachery? + Son. He ha's kill'd me Mother, +Run away I pray you. + +Exit crying Murther. + + +Scaena Tertia. + +Enter Malcolme and Macduffe. + + Mal. Let vs seeke out some desolate shade, & there +Weepe our sad bosomes empty + + Macd. Let vs rather +Hold fast the mortall Sword: and like good men, +Bestride our downfall Birthdome: each new Morne, +New Widdowes howle, new Orphans cry, new sorowes +Strike heauen on the face, that it resounds +As if it felt with Scotland, and yell'd out +Like Syllable of Dolour + + Mal. What I beleeue, Ile waile; +What know, beleeue; and what I can redresse, +As I shall finde the time to friend: I wil. +What you haue spoke, it may be so perchance. +This Tyrant, whose sole name blisters our tongues, +Was once thought honest: you haue lou'd him well, +He hath not touch'd you yet. I am yong, but something +You may discerne of him through me, and wisedome +To offer vp a weake, poore innocent Lambe +T' appease an angry God + + Macd. I am not treacherous + + Malc. But Macbeth is. +A good and vertuous Nature may recoyle +In an Imperiall charge. But I shall craue your pardon: +That which you are, my thoughts cannot transpose; +Angels are bright still, though the brightest fell. +Though all things foule, would wear the brows of grace +Yet Grace must still looke so + + Macd. I haue lost my Hopes + + Malc. Perchance euen there +Where I did finde my doubts. +Why in that rawnesse left you Wife, and Childe? +Those precious Motiues, those strong knots of Loue, +Without leaue-taking. I pray you, +Let not my Iealousies, be your Dishonors, +But mine owne Safeties: you may be rightly iust, +What euer I shall thinke + + Macd. Bleed, bleed poore Country, +Great Tyrrany, lay thou thy basis sure, +For goodnesse dare not check thee: wear y thy wrongs, +The Title, is affear'd. Far thee well Lord, +I would not be the Villaine that thou think'st, +For the whole Space that's in the Tyrants Graspe, +And the rich East to boot + + Mal. Be not offended: +I speake not as in absolute feare of you: +I thinke our Country sinkes beneath the yoake, +It weepes, it bleeds, and each new day a gash +Is added to her wounds. I thinke withall, +There would be hands vplifted in my right: +And heere from gracious England haue I offer +Of goodly thousands. But for all this, +When I shall treade vpon the Tyrants head, +Or weare it on my Sword; yet my poore Country +Shall haue more vices then it had before, +More suffer, and more sundry wayes then euer, +By him that shall succeede + + Macd. What should he be? + Mal. It is my selfe I meane: in whom I know +All the particulars of Vice so grafted, +That when they shall be open'd, blacke Macbeth +Will seeme as pure as Snow, and the poore State +Esteeme him as a Lambe, being compar'd +With my confinelesse harmes + + Macd. Not in the Legions +Of horrid Hell, can come a Diuell more damn'd +In euils, to top Macbeth + + Mal. I grant him Bloody, +Luxurious, Auaricious, False, Deceitfull, +Sodaine, Malicious, smacking of euery sinne +That ha's a name. But there's no bottome, none +In my Voluptuousnesse: Your Wiues, your Daughters, +Your Matrons, and your Maides, could not fill vp +The Cesterne of my Lust, and my Desire +All continent Impediments would ore-beare +That did oppose my will. Better Macbeth, +Then such an one to reigne + + Macd. Boundlesse intemperance +In Nature is a Tyranny: It hath beene +Th' vntimely emptying of the happy Throne, +And fall of many Kings. But feare not yet +To take vpon you what is yours: you may +Conuey your pleasures in a spacious plenty, +And yet seeme cold. The time you may so hoodwinke: +We haue willing Dames enough: there cannot be +That Vulture in you, to deuoure so many +As will to Greatnesse dedicate themselues, +Finding it so inclinde + + Mal. With this, there growes +In my most ill-composd Affection, such +A stanchlesse Auarice, that were I King, +I should cut off the Nobles for their Lands, +Desire his Iewels, and this others House, +And my more-hauing, would be as a Sawce +To make me hunger more, that I should forge +Quarrels vniust against the Good and Loyall, +Destroying them for wealth + + Macd. This Auarice +stickes deeper: growes with more pernicious roote +Then Summer-seeming Lust: and it hath bin +The Sword of our slaine Kings: yet do not feare, +Scotland hath Foysons, to fill vp your will +Of your meere Owne. All these are portable, +With other Graces weigh'd + + Mal. But I haue none. The King-becoming Graces, +As Iustice, Verity, Temp'rance, Stablenesse, +Bounty, Perseuerance, Mercy, Lowlinesse, +Deuotion, Patience, Courage, Fortitude, +I haue no rellish of them, but abound +In the diuision of each seuerall Crime, +Acting it many wayes. Nay, had I powre, I should +Poure the sweet Milke of Concord, into Hell, +Vprore the vniuersall peace, confound +All vnity on earth + + Macd. O Scotland, Scotland + + Mal. If such a one be fit to gouerne, speake: +I am as I haue spoken + + Mac. Fit to gouern? No not to liue. O Natio[n] miserable! +With an vntitled Tyrant, bloody Sceptred, +When shalt thou see thy wholsome dayes againe? +Since that the truest Issue of thy Throne +By his owne Interdiction stands accust, +And do's blaspheme his breed? Thy Royall Father +Was a most Sainted-King: the Queene that bore thee, +Oftner vpon her knees, then on her feet, +Dy'de euery day she liu'd. Fare thee well, +These Euils thou repeat'st vpon thy selfe, +Hath banish'd me from Scotland. O my Brest, +Thy hope ends heere + + Mal. Macduff, this Noble passion +Childe of integrity, hath from my soule +Wip'd the blacke Scruples, reconcil'd my thoughts +To thy good Truth, and Honor. Diuellish Macbeth, +By many of these traines, hath sought to win me +Into his power: and modest Wisedome pluckes me +From ouer-credulous hast: but God aboue +Deale betweene thee and me; For euen now +I put my selfe to thy Direction, and +Vnspeake mine owne detraction. Heere abiure +The taints, and blames I laide vpon my selfe, +For strangers to my Nature. I am yet +Vnknowne to Woman, neuer was forsworne, +Scarsely haue coueted what was mine owne. +At no time broke my Faith, would not betray +The Deuill to his Fellow, and delight +No lesse in truth then life. My first false speaking +Was this vpon my selfe. What I am truly +Is thine, and my poore Countries to command: +Whither indeed, before they heere approach +Old Seyward with ten thousand warlike men +Already at a point, was setting foorth: +Now wee'l together, and the chance of goodnesse +Be like our warranted Quarrell. Why are you silent? + Macd. Such welcome, and vnwelcom things at once +'Tis hard to reconcile. +Enter a Doctor. + + Mal. Well, more anon. Comes the King forth +I pray you? + Doct. I Sir: there are a crew of wretched Soules +That stay his Cure: their malady conuinces +The great assay of Art. But at his touch, +Such sanctity hath Heauen giuen his hand, +They presently amend. +Enter. + + Mal. I thanke you Doctor + + Macd. What's the Disease he meanes? + Mal. Tis call'd the Euill. +A most myraculous worke in this good King, +Which often since my heere remaine in England, +I haue seene him do: How he solicites heauen +Himselfe best knowes: but strangely visited people +All swolne and Vlcerous, pittifull to the eye, +The meere dispaire of Surgery, he cures, +Hanging a golden stampe about their neckes, +Put on with holy Prayers, and 'tis spoken +To the succeeding Royalty he leaues +The healing Benediction. With this strange vertue, +He hath a heauenly guift of Prophesie, +And sundry Blessings hang about his Throne, +That speake him full of Grace. +Enter Rosse. + + Macd. See who comes heere + + Malc. My Countryman: but yet I know him not + + Macd. My euer gentle Cozen, welcome hither + + Malc. I know him now. Good God betimes remoue +The meanes that makes vs Strangers + + Rosse. Sir, Amen + + Macd. Stands Scotland where it did? + Rosse. Alas poore Countrey, +Almost affraid to know it selfe. It cannot +Be call'd our Mother, but our Graue; where nothing +But who knowes nothing, is once seene to smile: +Where sighes, and groanes, and shrieks that rent the ayre +Are made, not mark'd: Where violent sorrow seemes +A Moderne extasie: The Deadmans knell, +Is there scarse ask'd for who, and good mens liues +Expire before the Flowers in their Caps, +Dying, or ere they sicken + + Macd. Oh Relation; too nice, and yet too true + + Malc. What's the newest griefe? + Rosse. That of an houres age, doth hisse the speaker, +Each minute teemes a new one + + Macd. How do's my Wife? + Rosse. Why well + + Macd. And all my Children? + Rosse. Well too + + Macd. The Tyrant ha's not batter'd at their peace? + Rosse. No, they were wel at peace, when I did leaue 'em + Macd. Be not a niggard of your speech: How gos't? + Rosse. When I came hither to transport the Tydings +Which I haue heauily borne, there ran a Rumour +Of many worthy Fellowes, that were out, +Which was to my beleefe witnest the rather, +For that I saw the Tyrants Power a-foot. +Now is the time of helpe: your eye in Scotland +Would create Soldiours, make our women fight, +To doffe their dire distresses + + Malc. Bee't their comfort +We are comming thither: Gracious England hath +Lent vs good Seyward, and ten thousand men, +An older, and a better Souldier, none +That Christendome giues out + + Rosse. Would I could answer +This comfort with the like. But I haue words +That would be howl'd out in the desert ayre, +Where hearing should not latch them + + Macd. What concerne they, +The generall cause, or is it a Fee-griefe +Due to some single brest? + Rosse. No minde that's honest +But in it shares some woe, though the maine part +Pertaines to you alone + + Macd. If it be mine +Keepe it not from me, quickly let me haue it + + Rosse. Let not your eares dispise my tongue for euer, +Which shall possesse them with the heauiest sound +that euer yet they heard + + Macd. Humh: I guesse at it + + Rosse. Your Castle is surpriz'd: your Wife, and Babes +Sauagely slaughter'd: To relate the manner +Were on the Quarry of these murther'd Deere +To adde the death of you + + Malc. Mercifull Heauen: +What man, ne're pull your hat vpon your browes: +Giue sorrow words; the griefe that do's not speake, +Whispers the o're-fraught heart, and bids it breake + + Macd. My Children too? + Ro. Wife, Children, Seruants, all that could be found + + Macd. And I must be from thence? My wife kil'd too? + Rosse. I haue said + + Malc. Be comforted. +Let's make vs Med'cines of our great Reuenge, +To cure this deadly greefe + + Macd. He ha's no Children. All my pretty ones? +Did you say All? Oh Hell-Kite! All? +What, All my pretty Chickens, and their Damme +At one fell swoope? + Malc. Dispute it like a man + + Macd. I shall do so: +But I must also feele it as a man; +I cannot but remember such things were +That were most precious to me: Did heauen looke on, +And would not take their part? Sinfull Macduff, +They were all strooke for thee: Naught that I am, +Not for their owne demerits, but for mine +Fell slaughter on their soules: Heauen rest them now + + Mal. Be this the Whetstone of your sword, let griefe +Conuert to anger: blunt not the heart, enrage it + + Macd. O I could play the woman with mine eyes, +And Braggart with my tongue. But gentle Heauens, +Cut short all intermission: Front to Front, +Bring thou this Fiend of Scotland, and my selfe +Within my Swords length set him, if he scape +Heauen forgiue him too + + Mal. This time goes manly: +Come go we to the King, our Power is ready, +Our lacke is nothing but our leaue. Macbeth +Is ripe for shaking, and the Powres aboue +Put on their Instruments: Receiue what cheere you may, +The Night is long, that neuer findes the Day. + +Exeunt. + +Actus Quintus. Scena Prima. + +Enter a Doctor of Physicke, and a Wayting Gentlewoman. + + Doct. I haue too Nights watch'd with you, but can +perceiue no truth in your report. When was it shee last +walk'd? + Gent. Since his Maiesty went into the Field, I haue +seene her rise from her bed, throw her Night-Gown vppon +her, vnlocke her Closset, take foorth paper, folde it, +write vpon't, read it, afterwards Seale it, and againe returne +to bed; yet all this while in a most fast sleepe + + Doct. A great perturbation in Nature, to receyue at +once the benefit of sleep, and do the effects of watching. +In this slumbry agitation, besides her walking, and other +actuall performances, what (at any time) haue you heard +her say? + Gent. That Sir, which I will not report after her + + Doct. You may to me, and 'tis most meet you should + + Gent. Neither to you, nor any one, hauing no witnesse +to confirme my speech. +Enter Lady, with a Taper. + +Lo you, heere she comes: This is her very guise, and vpon +my life fast asleepe: obserue her, stand close + + Doct. How came she by that light? + Gent. Why it stood by her: she ha's light by her continually, +'tis her command + + Doct. You see her eyes are open + + Gent. I, but their sense are shut + + Doct. What is it she do's now? +Looke how she rubbes her hands + + Gent. It is an accustom'd action with her, to seeme +thus washing her hands: I haue knowne her continue in +this a quarter of an houre + + Lad. Yet heere's a spot + + Doct. Heark, she speaks, I will set downe what comes +from her, to satisfie my remembrance the more strongly + + La. Out damned spot: out I say. One: Two: Why +then 'tis time to doo't: Hell is murky. Fye, my Lord, fie, +a Souldier, and affear'd? what need we feare? who knowes +it, when none can call our powre to accompt: yet who +would haue thought the olde man to haue had so much +blood in him + + Doct. Do you marke that? + Lad. The Thane of Fife, had a wife: where is she now? +What will these hands ne're be cleane? No more o'that +my Lord, no more o'that: you marre all with this starting + + Doct. Go too, go too: +You haue knowne what you should not + + Gent. She ha's spoke what shee should not, I am sure +of that: Heauen knowes what she ha's knowne + + La. Heere's the smell of the blood still: all the perfumes +of Arabia will not sweeten this little hand. +Oh, oh, oh + + Doct. What a sigh is there? The hart is sorely charg'd + + Gent. I would not haue such a heart in my bosome, +for the dignity of the whole body + + Doct. Well, well, well + + Gent. Pray God it be sir + + Doct. This disease is beyond my practise: yet I haue +knowne those which haue walkt in their sleep, who haue +dyed holily in their beds + + Lad. Wash your hands, put on your Night-Gowne, +looke not so pale: I tell you yet againe Banquo's buried; +he cannot come out on's graue + + Doct. Euen so? + Lady. To bed, to bed: there's knocking at the gate: +Come, come, come, come, giue me your hand: What's +done, cannot be vndone. To bed, to bed, to bed. + +Exit Lady. + + Doct. Will she go now to bed? + Gent. Directly + + Doct. Foule whisp'rings are abroad: vnnaturall deeds +Do breed vnnaturall troubles: infected mindes +To their deafe pillowes will discharge their Secrets: +More needs she the Diuine, then the Physitian: +God, God forgiue vs all. Looke after her, +Remoue from her the meanes of all annoyance, +And still keepe eyes vpon her: So goodnight, +My minde she ha's mated, and amaz'd my sight. +I thinke, but dare not speake + + Gent. Good night good Doctor. + +Exeunt. + + +Scena Secunda. + +Drum and Colours. Enter Menteth, Cathnes, Angus, Lenox, +Soldiers. + + Ment. The English powre is neere, led on by Malcolm, +His Vnkle Seyward, and the good Macduff. +Reuenges burne in them: for their deere causes +Would to the bleeding, and the grim Alarme +Excite the mortified man + + Ang. Neere Byrnan wood +Shall we well meet them, that way are they comming + + Cath. Who knowes if Donalbane be with his brother? + Len. For certaine Sir, he is not: I haue a File +Of all the Gentry; there is Seywards Sonne, +And many vnruffe youths, that euen now +Protest their first of Manhood + + Ment. What do's the Tyrant + + Cath. Great Dunsinane he strongly Fortifies: +Some say hee's mad: Others, that lesser hate him, +Do call it valiant Fury, but for certaine +He cannot buckle his distemper'd cause +Within the belt of Rule + + Ang. Now do's he feele +His secret Murthers sticking on his hands, +Now minutely Reuolts vpbraid his Faith-breach: +Those he commands, moue onely in command, +Nothing in loue: Now do's he feele his Title +Hang loose about him, like a Giants Robe +Vpon a dwarfish Theefe + + Ment. Who then shall blame +His pester'd Senses to recoyle, and start, +When all that is within him, do's condemne +It selfe, for being there + + Cath. Well, march we on, +To giue Obedience, where 'tis truly ow'd: +Meet we the Med'cine of the sickly Weale, +And with him poure we in our Countries purge, +Each drop of vs + + Lenox. Or so much as it needes, +To dew the Soueraigne Flower, and drowne the Weeds: +Make we our March towards Birnan. + +Exeunt. marching. + + +Scaena Tertia. + +Enter Macbeth, Doctor, and Attendants. + + Macb. Bring me no more Reports, let them flye all: +Till Byrnane wood remoue to Dunsinane, +I cannot taint with Feare. What's the Boy Malcolme? +Was he not borne of woman? The Spirits that know +All mortall Consequences, haue pronounc'd me thus: +Feare not Macbeth, no man that's borne of woman +Shall ere haue power vpon thee. Then fly false Thanes, +And mingle with the English Epicures, +The minde I sway by, and the heart I beare, +Shall neuer sagge with doubt, nor shake with feare. +Enter Seruant. + +The diuell damne thee blacke, thou cream-fac'd Loone: +Where got'st thou that Goose-looke + + Ser. There is ten thousand + + Macb. Geese Villaine? + Ser. Souldiers Sir + + Macb. Go pricke thy face, and ouer-red thy feare +Thou Lilly-liuer'd Boy. What Soldiers, Patch? +Death of thy Soule, those Linnen cheekes of thine +Are Counsailers to feare. What Soldiers Whay-face? + Ser. The English Force, so please you + + Macb. Take thy face hence. Seyton, I am sick at hart, +When I behold: Seyton, I say, this push +Will cheere me euer, or dis-eate me now. +I haue liu'd long enough: my way of life +Is falne into the Seare, the yellow Leafe, +And that which should accompany Old-Age, +As Honor, Loue, Obedience, Troopes of Friends, +I must not looke to haue: but in their steed, +Curses, not lowd but deepe, Mouth-honor, breath +Which the poore heart would faine deny, and dare not. +Seyton? +Enter Seyton. + + Sey. What's your gracious pleasure? + Macb. What Newes more? + Sey. All is confirm'd my Lord, which was reported + + Macb. Ile fight, till from my bones, my flesh be hackt. +Giue me my Armor + + Seyt. 'Tis not needed yet + + Macb. Ile put it on: +Send out moe Horses, skirre the Country round, +Hang those that talke of Feare. Giue me mine Armor: +How do's your Patient, Doctor? + Doct. Not so sicke my Lord, +As she is troubled with thicke-comming Fancies +That keepe her from her rest + + Macb. Cure of that: +Can'st thou not Minister to a minde diseas'd, +Plucke from the Memory a rooted Sorrow, +Raze out the written troubles of the Braine, +And with some sweet Obliuious Antidote +Cleanse the stufft bosome, of that perillous stuffe +Which weighes vpon the heart? + Doct. Therein the Patient +Must minister to himselfe + + Macb. Throw Physicke to the Dogs, Ile none of it. +Come, put mine Armour on: giue me my Staffe: +Seyton, send out: Doctor, the Thanes flye from me: +Come sir, dispatch. If thou could'st Doctor, cast +The Water of my Land, finde her Disease, +And purge it to a sound and pristine Health, +I would applaud thee to the very Eccho, +That should applaud againe. Pull't off I say, +What Rubarb, Cyme, or what Purgatiue drugge +Would scowre these English hence: hear'st y of them? + Doct. I my good Lord: your Royall Preparation +Makes vs heare something + + Macb. Bring it after me: +I will not be affraid of Death and Bane, +Till Birnane Forrest come to Dunsinane + + Doct. Were I from Dunsinane away, and cleere, +Profit againe should hardly draw me heere. + +Exeunt. + +Scena Quarta. + +Drum and Colours. Enter Malcolme, Seyward, Macduffe, +Seywards Sonne, +Menteth, Cathnes, Angus, and Soldiers Marching. + + Malc. Cosins, I hope the dayes are neere at hand +That Chambers will be safe + + Ment. We doubt it nothing + + Seyw. What wood is this before vs? + Ment. The wood of Birnane + + Malc. Let euery Souldier hew him downe a Bough, +And bear't before him, thereby shall we shadow +The numbers of our Hoast, and make discouery +Erre in report of vs + + Sold. It shall be done + + Syw. We learne no other, but the confident Tyrant +Keepes still in Dunsinane, and will indure +Our setting downe befor't + + Malc. 'Tis his maine hope: +For where there is aduantage to be giuen, +Both more and lesse haue giuen him the Reuolt, +And none serue with him, but constrained things, +Whose hearts are absent too + + Macd. Let our iust Censures +Attend the true euent, and put we on +Industrious Souldiership + + Sey. The time approaches, +That will with due decision make vs know +What we shall say we haue, and what we owe: +Thoughts speculatiue, their vnsure hopes relate, +But certaine issue, stroakes must arbitrate, +Towards which, aduance the warre. + +Exeunt. marching + +Scena Quinta. + +Enter Macbeth, Seyton, & Souldiers, with Drum and Colours. + + Macb. Hang out our Banners on the outward walls, +The Cry is still, they come: our Castles strength +Will laugh a Siedge to scorne: Heere let them lye, +Till Famine and the Ague eate them vp: +Were they not forc'd with those that should be ours, +We might haue met them darefull, beard to beard, +And beate them backward home. What is that noyse? + +A Cry within of Women. + + Sey. It is the cry of women, my good Lord + + Macb. I haue almost forgot the taste of Feares: +The time ha's beene, my sences would haue cool'd +To heare a Night-shrieke, and my Fell of haire +Would at a dismall Treatise rowze, and stirre +As life were in't. I haue supt full with horrors, +Direnesse familiar to my slaughterous thoughts +Cannot once start me. Wherefore was that cry? + Sey. The Queene (my Lord) is dead + + Macb. She should haue dy'de heereafter; +There would haue beene a time for such a word: +To morrow, and to morrow, and to morrow, +Creepes in this petty pace from day to day, +To the last Syllable of Recorded time: +And all our yesterdayes, haue lighted Fooles +The way to dusty death. Out, out, breefe Candle, +Life's but a walking Shadow, a poore Player, +That struts and frets his houre vpon the Stage, +And then is heard no more. It is a Tale +Told by an Ideot, full of sound and fury +Signifying nothing. +Enter a Messenger. + +Thou com'st to vse thy Tongue: thy Story quickly + + Mes. Gracious my Lord, +I should report that which I say I saw, +But know not how to doo't + + Macb. Well, say sir + + Mes. As I did stand my watch vpon the Hill +I look'd toward Byrnane, and anon me thought +The Wood began to moue + + Macb. Lyar, and Slaue + + Mes. Let me endure your wrath, if't be not so: +Within this three Mile may you see it comming. +I say, a mouing Groue + + Macb. If thou speak'st false, +Vpon the next Tree shall thou hang aliue +Till Famine cling thee: If thy speech be sooth, +I care not if thou dost for me as much. +I pull in Resolution, and begin +To doubt th' Equiuocation of the Fiend, +That lies like truth. Feare not, till Byrnane Wood +Do come to Dunsinane, and now a Wood +Comes toward Dunsinane. Arme, Arme, and out, +If this which he auouches, do's appeare, +There is nor flying hence, nor tarrying here. +I 'ginne to be a-weary of the Sun, +And wish th' estate o'th' world were now vndon. +Ring the Alarum Bell, blow Winde, come wracke, +At least wee'l dye with Harnesse on our backe. + +Exeunt. + +Scena Sexta. + +Drumme and Colours. Enter Malcolme, Seyward, Macduffe, and +their Army, +with Boughes. + + Mal. Now neere enough: +Your leauy Skreenes throw downe, +And shew like those you are: You (worthy Vnkle) +Shall with my Cosin your right Noble Sonne +Leade our first Battell. Worthy Macduffe, and wee +Shall take vpon's what else remaines to do, +According to our order + + Sey. Fare you well: +Do we but finde the Tyrants power to night, +Let vs be beaten, if we cannot fight + + Macd. Make all our Trumpets speak, giue the[m] all breath +Those clamorous Harbingers of Blood, & Death. + +Exeunt. + +Alarums continued. + + +Scena Septima. + +Enter Macbeth. + + Macb. They haue tied me to a stake, I cannot flye, +But Beare-like I must fight the course. What's he +That was not borne of Woman? Such a one +Am I to feare, or none. +Enter young Seyward. + + Y.Sey. What is thy name? + Macb. Thou'lt be affraid to heare it + + Y.Sey. No: though thou call'st thy selfe a hoter name +Then any is in hell + + Macb. My name's Macbeth + + Y.Sey. The diuell himselfe could not pronounce a Title +More hatefull to mine eare + + Macb. No: nor more fearefull + + Y.Sey. Thou lyest abhorred Tyrant, with my Sword +Ile proue the lye thou speak'st. + +Fight, and young Seyward slaine. + + Macb. Thou was't borne of woman; +But Swords I smile at, Weapons laugh to scorne, +Brandish'd by man that's of a Woman borne. +Enter. + +Alarums. Enter Macduffe. + + Macd. That way the noise is: Tyrant shew thy face, +If thou beest slaine, and with no stroake of mine, +My Wife and Childrens Ghosts will haunt me still: +I cannot strike at wretched Kernes, whose armes +Are hyr'd to beare their Staues; either thou Macbeth, +Or else my Sword with an vnbattered edge +I sheath againe vndeeded. There thou should'st be, +By this great clatter, one of greatest note +Seemes bruited. Let me finde him Fortune, +And more I begge not. + +Exit. Alarums. + +Enter Malcolme and Seyward. + + Sey. This way my Lord, the Castles gently rendred: +The Tyrants people, on both sides do fight, +The Noble Thanes do brauely in the Warre, +The day almost it selfe professes yours, +And little is to do + + Malc. We haue met with Foes +That strike beside vs + + Sey. Enter Sir, the Castle. + +Exeunt. Alarum + +Enter Macbeth. + + Macb. Why should I play the Roman Foole, and dye +On mine owne sword? whiles I see liues, the gashes +Do better vpon them. +Enter Macduffe. + + Macd. Turne Hell-hound, turne + + Macb. Of all men else I haue auoyded thee: +But get thee backe, my soule is too much charg'd +With blood of thine already + + Macd. I haue no words, +My voice is in my Sword, thou bloodier Villaine +Then tearmes can giue thee out. + +Fight: Alarum + + Macb. Thou loosest labour +As easie may'st thou the intrenchant Ayre +With thy keene Sword impresse, as make me bleed: +Let fall thy blade on vulnerable Crests, +I beare a charmed Life, which must not yeeld +To one of woman borne + + Macd. Dispaire thy Charme, +And let the Angell whom thou still hast seru'd +Tell thee, Macduffe was from his Mothers womb +Vntimely ript + + Macb. Accursed be that tongue that tels mee so; +For it hath Cow'd my better part of man: +And be these Iugling Fiends no more beleeu'd, +That palter with vs in a double sence, +That keepe the word of promise to our eare, +And breake it to our hope. Ile not fight with thee + + Macd. Then yeeld thee Coward, +And liue to be the shew, and gaze o'th' time. +Wee'l haue thee, as our rarer Monsters are +Painted vpon a pole, and vnder-writ, +Heere may you see the Tyrant + + Macb. I will not yeeld +To kisse the ground before young Malcolmes feet, +And to be baited with the Rabbles curse. +Though Byrnane wood be come to Dunsinane, +And thou oppos'd, being of no woman borne, +Yet I will try the last. Before my body, +I throw my warlike Shield: Lay on Macduffe, +And damn'd be him, that first cries hold, enough. + +Exeunt. fighting. Alarums. + +Enter Fighting, and Macbeth slaine. + +Retreat, and Flourish. Enter with Drumme and Colours, Malcolm, +Seyward, +Rosse, Thanes, & Soldiers. + + Mal. I would the Friends we misse, were safe arriu'd + + Sey. Some must go off: and yet by these I see, +So great a day as this is cheapely bought + + Mal. Macduffe is missing, and your Noble Sonne + + Rosse. Your son my Lord, ha's paid a souldiers debt, +He onely liu'd but till he was a man, +The which no sooner had his Prowesse confirm'd +In the vnshrinking station where he fought, +But like a man he dy'de + + Sey. Then he is dead? + Rosse. I, and brought off the field: your cause of sorrow +Must not be measur'd by his worth, for then +It hath no end + + Sey. Had he his hurts before? + Rosse. I, on the Front + + Sey. Why then, Gods Soldier be he: +Had I as many Sonnes, as I haue haires, +I would not wish them to a fairer death: +And so his Knell is knoll'd + + Mal. Hee's worth more sorrow, +and that Ile spend for him + + Sey. He's worth no more, +They say he parted well, and paid his score, +And so God be with him. Here comes newer comfort. +Enter Macduffe, with Macbeths head. + + Macd. Haile King, for so thou art. +Behold where stands +Th' Vsurpers cursed head: the time is free: +I see thee compast with thy Kingdomes Pearle, +That speake my salutation in their minds: +Whose voyces I desire alowd with mine. +Haile King of Scotland + + All. Haile King of Scotland. + +Flourish. + + Mal. We shall not spend a large expence of time, +Before we reckon with your seuerall loues, +And make vs euen with you. My Thanes and Kinsmen +Henceforth be Earles, the first that euer Scotland +In such an Honor nam'd: What's more to do, +Which would be planted newly with the time, +As calling home our exil'd Friends abroad, +That fled the Snares of watchfull Tyranny, +Producing forth the cruell Ministers +Of this dead Butcher, and his Fiend-like Queene; +Who (as 'tis thought) by selfe and violent hands, +Tooke off her life. This, and what need full else +That call's vpon vs, by the Grace of Grace, +We will performe in measure, time, and place: +So thankes to all at once, and to each one, +Whom we inuite, to see vs Crown'd at Scone. + +Flourish. Exeunt Omnes. + + +FINIS. THE TRAGEDIE OF MACBETH. From e1e9ba73e4e7ded5911ae5fcaa26eb7e385c24a6 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 12 May 2015 10:53:52 +0100 Subject: [PATCH 046/137] Make it possible to run the SDK word count example with the SparkPipelineRunner. --- runners/spark/README.md | 18 +++++++++++++ runners/spark/pom.xml | 27 +++++++++++++++++++ .../spark/SparkPipelineOptionsRegistrar.java | 27 +++++++++++++++++++ .../dataflow/spark/SparkPipelineRunner.java | 11 ++++++++ .../spark/SparkPipelineRunnerRegistrar.java | 27 +++++++++++++++++++ ...aflow.sdk.options.PipelineOptionsRegistrar | 16 +++++++++++ ...taflow.sdk.runners.PipelineRunnerRegistrar | 16 +++++++++++ 7 files changed, 142 insertions(+) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptionsRegistrar.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunnerRegistrar.java create mode 100644 runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar create mode 100644 runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar diff --git a/runners/spark/README.md b/runners/spark/README.md index 611c062e50b4..7dceba6befbf 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -57,3 +57,21 @@ would do the following: options.setSparkMaster("spark://host:port"); EvaluationResult result = SparkPipelineRunner.create(options).run(p); +## Word Count Example + +First download a text document to use as input: + + curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > /tmp/kinglear.txt + +Then run the [word count example][wc] from the SDK using a single threaded Spark instance +in local mode: + + mvn exec:exec -Dclass=com.google.cloud.dataflow.examples.WordCount \ + -Dinput=/tmp/kinglear.txt -Doutput=/tmp/out -Drunner=SparkPipelineRunner \ + -DsparkMaster=local + +Check the output by running: + + head /tmp/out/part-00000 + +[wc]: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java \ No newline at end of file diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 33ec34ed4680..b894affd41ef 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,6 +22,11 @@ License. UTF-8 UTF-8 1.7 + com.google.cloud.dataflow.examples.WordCount + /tmp/kinglear.txt + /tmp/out + SparkPipelineRunner + local @@ -171,6 +176,23 @@ License. + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + java + + -classpath + + ${class} + --input=${input} + --output=${output} + --runner=${runner} + --sparkMaster=${sparkMaster} + + + @@ -205,6 +227,11 @@ License. google-cloud-dataflow-java-sdk-all 0.3.150326 + + com.google.cloud.dataflow + google-cloud-dataflow-java-examples-all + 0.3.150326 + org.apache.avro avro-mapred 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 new file mode 100644 index 000000000000..21fe6932990f --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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> getPipelineOptions() { + return ImmutableList.>of(SparkPipelineOptions.class); + } +} 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 index 654b7e289927..876383f0ecda 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -16,6 +16,9 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; +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; @@ -76,6 +79,14 @@ public static SparkPipelineRunner create(SparkPipelineOptions options) { return new SparkPipelineRunner(options); } + /** + * Constructs a SparkPipelineRunner from the given 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 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 new file mode 100644 index 000000000000..5bdd3222bfbd --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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>> getPipelineRunners() { + return ImmutableList.>>of(SparkPipelineRunner.class); + } +} 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 new file mode 100644 index 000000000000..045d5dd0cfb1 --- /dev/null +++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar @@ -0,0 +1,16 @@ +# +# Copyright 2014 Cloudera Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this 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. +# +com.cloudera.dataflow.spark.SparkPipelineOptionsRegistrar \ No newline at end of file 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 new file mode 100644 index 000000000000..26e0b3a19809 --- /dev/null +++ b/runners/spark/src/main/resources/META-INF/services/com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar @@ -0,0 +1,16 @@ +# +# Copyright 2014 Cloudera Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this 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. +# +com.cloudera.dataflow.spark.SparkPipelineRunnerRegistrar \ No newline at end of file From e5deb38a83380ca16a3bcad3abf685a453eb2070 Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 13 May 2015 12:26:16 +0100 Subject: [PATCH 047/137] Change to use SLF4J rather than java.util.logging. --- .../dataflow/spark/BroadcastHelper.java | 12 +++++------ .../cloudera/dataflow/spark/DoFnFunction.java | 15 +++++++------ .../dataflow/spark/SparkPipelineRunner.java | 21 ++++++++----------- .../dataflow/spark/TransformTranslator.java | 13 ++++++------ 4 files changed, 28 insertions(+), 33 deletions(-) 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 index 75c73a43c396..1fcb503c2f03 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -16,16 +16,16 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.coders.Coder; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.broadcast.Broadcast; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.Serializable; -import java.util.logging.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class BroadcastHelper implements Serializable { - private static Logger LOG = Logger.getLogger(BroadcastHelper.class.getName()); + private static Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); private Broadcast bcast; private final Coder coder; private transient T value; @@ -52,7 +52,7 @@ private T deserialize() { 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.warning(ioe.getMessage()); + LOG.warn(ioe.getMessage()); val = null; } return val; 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 index 489f0a7fb3ee..e84799d80820 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -21,19 +21,18 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.util.AggregatorImpl; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.joda.time.Instant; - import java.util.Collection; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.logging.Logger; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Dataflow's Do functions correspond to Spark's FlatMap functions. @@ -42,7 +41,7 @@ * @param Output element type. */ class DoFnFunction implements FlatMapFunction, O> { - private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; @@ -107,7 +106,7 @@ public synchronized void output(O o) { public void sideOutput(TupleTag tupleTag, T t) { String message = "sideOutput is an unsupported operation for doFunctions, use a " + "MultiDoFunction instead."; - LOG.warning(message); + LOG.warn(message); throw new UnsupportedOperationException(message); } @@ -115,7 +114,7 @@ public void sideOutput(TupleTag tupleTag, T t) { public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { String message = "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " + "MultiDoFunction instead."; - LOG.warning(message); + LOG.warn(message); throw new UnsupportedOperationException(message); } 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 index 876383f0ecda..ab6f1a8085de 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -16,7 +16,6 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; @@ -25,9 +24,9 @@ import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; - -import java.util.logging.Logger; import org.apache.spark.serializer.KryoSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The SparkPipelineRunner translate operations defined on a pipeline to a representation @@ -52,7 +51,7 @@ */ public class SparkPipelineRunner extends PipelineRunner { - private static final Logger LOG = Logger.getLogger(SparkPipelineRunner.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(SparkPipelineRunner.class); /** * Options used in this pipeline runner. */ @@ -143,10 +142,9 @@ public void enterCompositeTransform(TransformTreeNode node) { if (node.getTransform() != null && TransformTranslator.hasTransformEvaluator(node.getTransform().getClass())) { - LOG.info(String.format( - "Entering directly-translatable composite transform: '%s'", node.getFullName())); - LOG.fine(String.format( - "Composite transform class: '%s'", node.getTransform().getClass())); + LOG.info("Entering directly-translatable composite transform: '{}'", + node.getFullName()); + LOG.debug("Composite transform class: '{}'", node.getTransform().getClass()); currentTranslatedCompositeNode = node; } } @@ -157,8 +155,8 @@ public void leaveCompositeTransform(TransformTreeNode 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(String.format( - "Post-visiting directly-translatable composite transform: '%s'", node.getFullName())); + LOG.info("Post-visiting directly-translatable composite transform: '{}'", + node.getFullName()); doVisitTransform(node.getTransform()); currentTranslatedCompositeNode = null; } @@ -167,8 +165,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { @Override public void visitTransform(TransformTreeNode node) { if (inTranslatedCompositeNode()) { - LOG.info(String.format( - "Skipping '%s'; already in composite transform.", node.getFullName())); + LOG.info("Skipping '{}'; already in composite transform.", node.getFullName()); return; } doVisitTransform(node.getTransform()); 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 index 5f4a721e1b02..94d09b695320 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -37,6 +37,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import java.io.IOException; +import java.lang.reflect.Field; +import java.util.List; +import java.util.Map; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; @@ -49,20 +52,16 @@ import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; -import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; -import java.lang.reflect.Field; -import java.util.List; -import java.util.Map; -import java.util.logging.Logger; - /** * Supports translation between a DataFlow transform, and Spark's operations on RDDs. */ public final class TransformTranslator { - private static final Logger LOG = Logger.getLogger(TransformTranslator.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(TransformTranslator.class); private TransformTranslator() { } From ab1503b522e80c235860c65d0dd9afb84112faaa Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 13 May 2015 12:28:00 +0100 Subject: [PATCH 048/137] Exclude jul-to-slf4j to avoid loops. --- runners/spark/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index b894affd41ef..976df61c2ddd 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -216,6 +216,14 @@ License. org.apache.spark spark-core_2.10 1.2.0 + + + + org.slf4j + jul-to-slf4j + + com.google.guava From 4fa925c5e5af907414890b60a7c153f56643fd43 Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 13 May 2015 15:06:43 +0100 Subject: [PATCH 049/137] Exclude old Jetty version of servlet API. --- runners/spark/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 976df61c2ddd..81d993045b3e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -245,6 +245,13 @@ License. avro-mapred 1.7.7 hadoop2 + + + + org.mortbay.jetty + servlet-api + + From 24bad5e710dc29a877c8872ce5df51515ed59b3c Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 13 May 2015 15:19:11 +0100 Subject: [PATCH 050/137] Update to dataflow 0.4.150414. --- runners/spark/pom.xml | 4 ++-- .../com/cloudera/dataflow/spark/DoFnFunction.java | 12 ++++++++---- .../cloudera/dataflow/spark/EvaluationContext.java | 6 ++++++ .../cloudera/dataflow/spark/MultiDoFnFunction.java | 12 ++++++++---- 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 81d993045b3e..b082c457ef9c 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -233,12 +233,12 @@ License. com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - 0.3.150326 + 0.4.150414 com.google.cloud.dataflow google-cloud-dataflow-java-examples-all - 0.3.150326 + 0.4.150414 org.apache.avro 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 index e84799d80820..12458d254fd1 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -21,10 +21,9 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; -import com.google.common.collect.ImmutableList; -import java.util.Collection; import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -153,8 +152,13 @@ public Instant timestamp() { } @Override - public Collection windows() { - return ImmutableList.of(); + public BoundedWindow window() { + return null; + } + + @Override + public WindowingInternals windowingInternals() { + return null; } } } 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 index bbcd053545bd..aecc1c559a57 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -16,6 +16,7 @@ package com.cloudera.dataflow.spark; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.coders.IterableCoder; @@ -177,4 +178,9 @@ public T apply(byte[] bytes) { public void close() { jsc.stop(); } + + @Override + public State getState() { + return State.UNKNOWN; + } } 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 index adcb4a264ec2..cc62a3223f7b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -15,7 +15,7 @@ package com.cloudera.dataflow.spark; -import java.util.Collection; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; import java.util.Iterator; import java.util.Map; @@ -28,7 +28,6 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; @@ -153,8 +152,13 @@ public Instant timestamp() { } @Override - public Collection windows() { - return ImmutableList.of(); + public BoundedWindow window() { + return null; + } + + @Override + public WindowingInternals windowingInternals() { + return null; } } } From ddaa532b0db57600fca25b1aac5f22459440020c Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 13 May 2015 16:09:16 +0100 Subject: [PATCH 051/137] Use DataflowAssert in tests. --- .../dataflow/spark/TransformTranslator.java | 16 +++++++++++ .../cloudera/dataflow/spark/DeDupTest.java | 19 +++++-------- .../dataflow/spark/SimpleWordCountTest.java | 28 ++++++++----------- 3 files changed, 34 insertions(+), 29 deletions(-) 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 index 94d09b695320..59e3a695867e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -26,6 +26,8 @@ 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.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -343,6 +345,19 @@ public Tuple2, NullWritable> call(T t) throws Exception { }; } + private static TransformEvaluator> window() { + return new TransformEvaluator>() { + @Override + public void evaluate(Window.Bound transform, EvaluationContext context) { + if (transform.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) { + context.setOutputRDD(transform, context.getInputRDD(transform)); + } else { + throw new UnsupportedOperationException("Non-global windowing not supported"); + } + } + }; + } + private static TransformEvaluator> create() { return new TransformEvaluator>() { @Override @@ -452,6 +467,7 @@ private static Map, BroadcastHelper> getSideInputs( mEvaluators.put(View.AsSingleton.class, viewAsSingleton()); mEvaluators.put(View.AsIterable.class, viewAsIter()); mEvaluators.put(View.CreatePCollectionView.class, createPCollView()); + mEvaluators.put(Window.Bound.class, window()); } public static boolean hasTransformEvaluator(Class clazz) { 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 index e26a50854995..8eb6e55b647e 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java @@ -2,16 +2,14 @@ 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.RemoveDuplicates; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.collect.ImmutableSet; import java.util.Arrays; -import java.util.HashSet; import java.util.List; import java.util.Set; -import org.junit.Assert; import org.junit.Test; /** @@ -28,17 +26,14 @@ public class DeDupTest { @Test public void testRun() throws Exception { - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + options.setRunner(SparkPipelineRunner.class); + Pipeline p = Pipeline.create(options); PCollection input = p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of()); PCollection output = input.apply(RemoveDuplicates.create()); - EvaluationResult res = SparkPipelineRunner.create().run(p); - Set actualSet = new HashSet<>(); - for (String s : res.get(output)) { - actualSet.add(s); - } - Assert.assertEquals(String.format("Actual lines [%s] does not equal expected [%s].", - actualSet, EXPECTED_SET), EXPECTED_SET, actualSet); - res.close(); + DataflowAssert.that(output).containsInAnyOrder(EXPECTED_SET); + + p.run(); } } 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 index 2628a6fa582a..df403eb09680 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -17,7 +17,7 @@ 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.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.Create; @@ -28,13 +28,10 @@ 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.Assert; -import org.junit.Test; - import java.util.Arrays; -import java.util.HashSet; import java.util.List; import java.util.Set; +import org.junit.Test; public class SimpleWordCountTest { private static final String[] WORDS_ARRAY = { @@ -46,20 +43,17 @@ public class SimpleWordCountTest { @Test public void testRun() throws Exception { - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + options.setRunner(SparkPipelineRunner.class); + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder + .of()); PCollection output = inputWords.apply(new CountWords()); - EvaluationResult res = SparkPipelineRunner.create().run(p); - Set actualCountSet = new HashSet<>(); - for (String s : res.get(output)) { - actualCountSet.add(s); - } - Assert.assertEquals(String.format("Actual counts of words [%s] does not equal expected " + - "count[%s].", - actualCountSet, EXPECTED_COUNT_SET), - EXPECTED_COUNT_SET, actualCountSet); - res.close(); + DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + p.run(); + } /** From aed5e276726440cb3cfa04fe6d16985aa7d2fb4f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 May 2015 11:37:28 +0100 Subject: [PATCH 052/137] Add Travis, Jacoco / Codecov integration. Fix Spark errors due to reused test JVM. --- runners/spark/.travis.yml | 19 ++++++++++++++ runners/spark/pom.xml | 52 ++++++++++++++++++++++++++++++--------- 2 files changed, 59 insertions(+), 12 deletions(-) create mode 100644 runners/spark/.travis.yml diff --git a/runners/spark/.travis.yml b/runners/spark/.travis.yml new file mode 100644 index 000000000000..2eeae9d1e69d --- /dev/null +++ b/runners/spark/.travis.yml @@ -0,0 +1,19 @@ +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, and code coverage + - jdk: openjdk7 + env: JACOCO=-Pjacoco + # Covers Spark 1.3 + - jdk: openjdk7 + env: SPARK=-Dspark.version=1.3.1 + # Covers Java 8, Oracle JDK + - jdk: oraclejdk8 + env: JAVA=-Djava.version=1.8 +cache: + directories: + - $HOME/.m2 +after_success: bash <(curl -s https://codecov.io/bash) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index b082c457ef9c..77e12aea927b 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,6 +22,7 @@ License. UTF-8 UTF-8 1.7 + 1.2.2 com.google.cloud.dataflow.examples.WordCount /tmp/kinglear.txt /tmp/out @@ -35,7 +36,7 @@ License. org.apache.maven.plugins maven-compiler-plugin - 3.2 + 3.3 ${java.version} ${java.version} @@ -49,17 +50,14 @@ License. maven-surefire-plugin 2.18.1 - - - true - + 1 + false org.apache.maven.plugins maven-checkstyle-plugin - 2.13 + 2.15 build-resources/header-file.txt build-resources/checkstyle.xml @@ -81,7 +79,7 @@ License. org.apache.maven.plugins maven-javadoc-plugin - 2.10.1 + 2.10.3 ${java.version} @@ -120,7 +118,7 @@ License. org.apache.maven.plugins maven-jar-plugin - 2.5 + 2.6 org.apache.maven.plugins @@ -130,7 +128,7 @@ License. org.apache.maven.plugins maven-release-plugin - 2.5.1 + 2.5.2 org.apache.maven.scm @@ -179,7 +177,7 @@ License. org.codehaus.mojo exec-maven-plugin - 1.2.1 + 1.4.0 java @@ -193,6 +191,25 @@ License. + + org.jacoco + jacoco-maven-plugin + 0.7.4.201502262128 + + + + prepare-agent + + + + report + test + + report + + + + @@ -215,7 +232,7 @@ License. org.apache.spark spark-core_2.10 - 1.2.0 + ${spark.version} @@ -359,6 +376,17 @@ License. + + jacoco + + + + org.jacoco + jacoco-maven-plugin + + + + From 3962b8241f1d5bd913b31db7a82ab25708df5f90 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 May 2015 13:52:24 +0100 Subject: [PATCH 053/137] Enable checkstyle, remove some rules, comply with checkstyle rules, fix some compiler warnings, fix some other items from IJ inspection --- runners/spark/build-resources/checkstyle.xml | 14 +-- runners/spark/pom.xml | 9 ++ .../dataflow/spark/BroadcastHelper.java | 7 +- .../cloudera/dataflow/spark/CoderHelpers.java | 2 + .../cloudera/dataflow/spark/DoFnFunction.java | 16 ++-- .../dataflow/spark/EvaluationContext.java | 26 ++---- .../dataflow/spark/MultiDoFnFunction.java | 2 +- .../dataflow/spark/SparkPipelineRunner.java | 13 +-- .../dataflow/spark/SparkRuntimeContext.java | 41 ++++----- .../dataflow/spark/TransformTranslator.java | 83 +++++++++--------- .../spark/aggregators/NamedAggregators.java | 85 ++++++++++--------- .../dataflow/spark/AvroPipelineTest.java | 32 ++++--- .../dataflow/spark/CombinePerKeyTest.java | 11 ++- .../cloudera/dataflow/spark/TfIdfTest.java | 64 +++++--------- .../spark/TransformTranslatorTest.java | 25 +++--- 15 files changed, 218 insertions(+), 212 deletions(-) diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml index 4800f3c57fc5..716270a6f377 100644 --- a/runners/spark/build-resources/checkstyle.xml +++ b/runners/spark/build-resources/checkstyle.xml @@ -61,15 +61,18 @@ - + - + + + + @@ -84,16 +87,13 @@ - - - - + @@ -128,7 +128,7 @@ value="BNOT, DEC, DOT, INC, LNOT, UNARY_MINUS, UNARY_PLUS"/> - + diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 77e12aea927b..926962482252 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -62,6 +62,15 @@ License. build-resources/header-file.txt build-resources/checkstyle.xml + + + validate + validate + + check + + + org.apache.maven.plugins 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 index 1fcb503c2f03..6a26787f320e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -15,17 +15,20 @@ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.coders.Coder; 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; class BroadcastHelper implements Serializable { - private static Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); + + private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); + private Broadcast bcast; private final Coder coder; private transient T value; 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 index 0a2dfaefd42d..4bf81e57cd10 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -36,6 +36,7 @@ private CoderHelpers() { * * @param value Value to serialize. * @param coder Coder to serialize with. + * @param type of value that is serialized * @return Byte array representing serialized object. */ static byte[] toByteArray(T value, Coder coder) { @@ -53,6 +54,7 @@ static byte[] toByteArray(T value, Coder coder) { * * @param values Values to serialize. * @param coder Coder to serialize with. + * @param type of value that is serialized * @return List of bytes representing serialized objects. */ static List toByteArrays(Iterable values, Coder coder) { 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 index 12458d254fd1..b02b87ad99ec 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -15,6 +15,11 @@ 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.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -24,10 +29,6 @@ import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; import org.slf4j.Logger; @@ -80,7 +81,7 @@ private class ProcCtxt extends DoFn.ProcessContext { private final List outputs = new LinkedList<>(); private I element; - public ProcCtxt(DoFn fn) { + ProcCtxt(DoFn fn) { fn.super(); } @@ -111,8 +112,9 @@ public void sideOutput(TupleTag tupleTag, T t) { @Override public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { - String message = "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " + - "MultiDoFunction instead."; + String message = + "sideOutputWithTimestamp is an unsupported operation for doFunctions, use a " + + "MultiDoFunction instead."; LOG.warn(message); throw new UnsupportedOperationException(message); } 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 index aecc1c559a57..24ecdeb2aeda 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -15,8 +15,13 @@ package com.cloudera.dataflow.spark; +import java.util.HashMap; +import java.util.HashSet; +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.PipelineResult; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.coders.IterableCoder; @@ -32,14 +37,8 @@ import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** - * Evaluation context allows us to define how pipeline instructions + * Evaluation context allows us to define how pipeline instructions. */ public class EvaluationContext implements EvaluationResult { private final JavaSparkContext jsc; @@ -86,12 +85,6 @@ Coder getDefaultCoder(T example) { return defaultCoder; } - /** - * Coder> getDefaultIterableCoder(Iterables iter) { - *

- * } - */ - I getInput(PTransform transform) { @SuppressWarnings("unchecked") I input = (I) pipeline.getInput(transform); @@ -133,8 +126,7 @@ void setRDD(PValue pvalue, JavaRDDLike rdd) { Iterable> getPCollectionView(PCollectionView view) { - Iterable> value = pview.get(view); - return value; + return pview.get(view); } @Override @@ -169,7 +161,7 @@ public Iterable get(PCollection pcollection) { return Iterables.transform(clientBytes, new Function() { @Override public T apply(byte[] bytes) { - return (T) CoderHelpers.fromByteArray(bytes, coder); + return CoderHelpers.fromByteArray(bytes, coder); } }); } 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 index cc62a3223f7b..37ddffb7ae7e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -15,7 +15,6 @@ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.sdk.util.WindowingInternals; import java.util.Iterator; import java.util.Map; @@ -25,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; 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 index ab6f1a8085de..b2e5150f2be8 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -49,7 +49,7 @@ * EvaluationResult result = SparkPipelineRunner.create(options).run(p); * } */ -public class SparkPipelineRunner extends PipelineRunner { +public final class SparkPipelineRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(SparkPipelineRunner.class); /** @@ -79,7 +79,10 @@ public static SparkPipelineRunner create(SparkPipelineOptions options) { } /** - * Constructs a SparkPipelineRunner from the given 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 = @@ -113,13 +116,13 @@ private JavaSparkContext getContext() { return new JavaSparkContext(conf); } - private static class Evaluator implements Pipeline.PipelineVisitor { + private static final class Evaluator implements Pipeline.PipelineVisitor { private final EvaluationContext ctxt; // Set upon entering a composite node which can be directly mapped to a single // TransformEvaluator. - private TransformTreeNode currentTranslatedCompositeNode = null; + private TransformTreeNode currentTranslatedCompositeNode; private Evaluator(EvaluationContext ctxt) { this.ctxt = ctxt; @@ -175,7 +178,7 @@ private void doVisitTransform(PT transform) { @SuppressWarnings("unchecked") TransformEvaluator evaluator = (TransformEvaluator) TransformTranslator.getTransformEvaluator(transform.getClass()); - LOG.info("Evaluating " + transform); + LOG.info("Evaluating {}", transform); evaluator.evaluate(transform, ctxt); } 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 index 8c721a83a5c4..c7bdc560b958 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -29,7 +29,6 @@ import com.google.cloud.dataflow.sdk.transforms.Min; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Sum; -import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.common.reflect.TypeToken; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; @@ -78,17 +77,17 @@ public synchronized PipelineOptions getPipelineOptions() { * * @param named Name of aggregator. * @param sfunc Serializable function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. + * @param Type of inputs to aggregator. + * @param Type of aggregator outputs. * @return Specified aggregator */ - public synchronized Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, - SerializableFunction, Out> sfunc) { + SerializableFunction, OUT> sfunc) { @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { - NamedAggregators.SerFunctionState state = new NamedAggregators + NamedAggregators.SerFunctionState state = new NamedAggregators .SerFunctionState<>(sfunc); accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator<>(state); @@ -102,19 +101,23 @@ public synchronized Aggregator createAggregator( * * @param named Name of aggregator. * @param combineFn Combine function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. + * @param Type of inputs to aggregator. + * @param Intermediate data type + * @param Type of aggregator outputs. * @return Specified aggregator */ - public synchronized Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, - Combine.CombineFn combineFn) { + Combine.CombineFn combineFn) { @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { @SuppressWarnings("unchecked") - NamedAggregators.CombineFunctionState state = new NamedAggregators - .CombineFunctionState<>((Combine.CombineFn) combineFn, (Coder) getCoder(combineFn), this); + NamedAggregators.CombineFunctionState state = + new NamedAggregators.CombineFunctionState<>( + (Combine.CombineFn) combineFn, + (Coder) getCoder(combineFn), + this); accum.add(new NamedAggregators(named, state)); aggregator = new SparkAggregator<>(state); aggregators.put(named, aggregator); @@ -158,17 +161,17 @@ private Coder getCoder(Combine.CombineFn combiner) { /** * Initialize spark aggregators exactly once. * - * @param Type of element fed in to aggregator. + * @param Type of element fed in to aggregator. */ - private static class SparkAggregator implements Aggregator, Serializable { - private final NamedAggregators.State state; + private static class SparkAggregator implements Aggregator, Serializable { + private final NamedAggregators.State state; - SparkAggregator(NamedAggregators.State state) { + SparkAggregator(NamedAggregators.State state) { this.state = state; } @Override - public void addValue(In elem) { + public void addValue(IN elem) { state.update(elem); } } 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 index 59e3a695867e..4030395d8a9c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -15,6 +15,11 @@ package com.cloudera.dataflow.spark; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.List; +import java.util.Map; + import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.AvroIO; @@ -38,10 +43,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.List; -import java.util.Map; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; @@ -52,18 +53,16 @@ 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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; import scala.Tuple2; /** * Supports translation between a DataFlow transform, and Spark's operations on RDDs. */ public final class TransformTranslator { - private static final Logger LOG = LoggerFactory.getLogger(TransformTranslator.class); private TransformTranslator() { } @@ -305,13 +304,18 @@ private static TransformEvaluator> readAvro() { @Override public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext() - .newAPIHadoopFile(pattern, AvroKeyInputFormat.class, AvroKey.class, - NullWritable.class, new Configuration()) - .map(new Function, Object>() { + JavaSparkContext jsc = context.getSparkContext(); + @SuppressWarnings("unchecked") + JavaRDD> avroFile = (JavaRDD>) (JavaRDD) + jsc.newAPIHadoopFile(pattern, + AvroKeyInputFormat.class, + AvroKey.class, NullWritable.class, + new Configuration()).keys(); + JavaRDD rdd = avroFile.map( + new Function, T>() { @Override - public Object call(Tuple2 t) throws Exception { - return t._1().datum(); + public T call(AvroKey key) { + return key.datum(); } }); context.setOutputRDD(transform, rdd); @@ -323,14 +327,14 @@ private static TransformEvaluator> writeAvro() { return new TransformEvaluator>() { @Override public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - @SuppressWarnings("unchecked") String pattern = transform.getFilenamePrefix(); + @SuppressWarnings("unchecked") JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); Job job; try { job = Job.getInstance(); } catch (IOException e) { - throw new RuntimeException(e); + throw new IllegalStateException(e); } AvroJob.setOutputKeySchema(job, transform.getSchema()); last.mapToPair(new PairFunction, NullWritable>() { @@ -398,12 +402,9 @@ public void evaluate(View.AsIterable transform, EvaluationContext context) { private static TransformEvaluator> createPCollView() { return new TransformEvaluator>() { @Override - public void evaluate(View.CreatePCollectionView transform, EvaluationContext - context) { - Iterable> iter = Iterables.transform(context.get(context.getInput - (transform)), new WindowingFunction() - ); - + public void evaluate(View.CreatePCollectionView transform, EvaluationContext context) { + Iterable> iter = Iterables.transform( + context.get(context.getInput(transform)), new WindowingFunction()); context.setPView(context.getOutput(transform), iter); } }; @@ -417,7 +418,9 @@ public WindowedValue apply(R t) { } } - private static class TupleTagFilter implements Function, Object>, Boolean> { + private static final class TupleTagFilter + implements Function, Object>, Boolean> { + private final TupleTag tag; private TupleTagFilter(TupleTag tag) { @@ -449,35 +452,35 @@ private static Map, BroadcastHelper> getSideInputs( } } - private static final Map, TransformEvaluator> mEvaluators = Maps + private static final Map, TransformEvaluator> EVALUATORS = Maps .newHashMap(); static { - mEvaluators.put(TextIO.Read.Bound.class, readText()); - mEvaluators.put(TextIO.Write.Bound.class, writeText()); - mEvaluators.put(AvroIO.Read.Bound.class, readAvro()); - mEvaluators.put(AvroIO.Write.Bound.class, writeAvro()); - mEvaluators.put(ParDo.Bound.class, parDo()); - mEvaluators.put(ParDo.BoundMulti.class, multiDo()); - mEvaluators.put(GroupByKey.GroupByKeyOnly.class, gbk()); - mEvaluators.put(Combine.GroupedValues.class, grouped()); - mEvaluators.put(Combine.PerKey.class, combinePerKey()); - mEvaluators.put(Flatten.FlattenPCollectionList.class, flattenPColl()); - mEvaluators.put(Create.class, create()); - mEvaluators.put(View.AsSingleton.class, viewAsSingleton()); - mEvaluators.put(View.AsIterable.class, viewAsIter()); - mEvaluators.put(View.CreatePCollectionView.class, createPCollView()); - mEvaluators.put(Window.Bound.class, window()); + 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(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.PerKey.class, combinePerKey()); + EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl()); + EVALUATORS.put(Create.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 boolean hasTransformEvaluator(Class clazz) { - return mEvaluators.containsKey(clazz); + return EVALUATORS.containsKey(clazz); } public static TransformEvaluator getTransformEvaluator(Class clazz) { @SuppressWarnings("unchecked") - TransformEvaluator transform = (TransformEvaluator) mEvaluators.get(clazz); + TransformEvaluator transform = (TransformEvaluator) EVALUATORS.get(clazz); if (transform == null) { throw new IllegalStateException("No TransformEvaluator registered for " + clazz); } 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 index 28db4d9d0f2f..904f71eebb3e 100644 --- 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 @@ -15,13 +15,6 @@ package com.cloudera.dataflow.spark.aggregators; -import com.cloudera.dataflow.spark.SparkRuntimeContext; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.CoderRegistry; -import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; @@ -29,6 +22,13 @@ import java.util.Map; import java.util.TreeMap; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +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. @@ -108,36 +108,36 @@ public String toString() { } /** - * @param Input data type - * @param Intermediate data type (useful for averages) - * @param Output data type + * @param Input data type + * @param Intermediate data type (useful for averages) + * @param Output data type */ - public interface State extends Serializable { + public interface State extends Serializable { /** * @param element new element to update state */ - void update(In element); + void update(IN element); - State merge(State other); + State merge(State other); - Inter current(); + INTER current(); - Out render(); + OUT render(); } /** * => combineFunction in data flow. */ - public static class CombineFunctionState implements State { + public static class CombineFunctionState implements State { - private Combine.CombineFn combineFn; - private Coder inCoder; + private Combine.CombineFn combineFn; + private Coder inCoder; private SparkRuntimeContext ctxt; - private transient Inter state; + private transient INTER state; public CombineFunctionState( - Combine.CombineFn combineFn, - Coder inCoder, + Combine.CombineFn combineFn, + Coder inCoder, SparkRuntimeContext ctxt) { this.combineFn = combineFn; this.inCoder = inCoder; @@ -146,23 +146,23 @@ public CombineFunctionState( } @Override - public void update(In element) { + public void update(IN element) { combineFn.addInput(state, element); } @Override - public State merge(State other) { + public State merge(State other) { this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); return this; } @Override - public Inter current() { + public INTER current() { return state; } @Override - public Out render() { + public OUT render() { return combineFn.extractOutput(state); } @@ -170,55 +170,58 @@ private void writeObject(ObjectOutputStream oos) throws IOException { oos.writeObject(ctxt); oos.writeObject(combineFn); oos.writeObject(inCoder); - combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder).encode(state, oos, Coder.Context.NESTED); + combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) + .encode(state, oos, Coder.Context.NESTED); } + @SuppressWarnings("unchecked") private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { ctxt = (SparkRuntimeContext) ois.readObject(); - combineFn = (Combine.CombineFn) ois.readObject(); - inCoder = (Coder) ois.readObject(); - state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder).decode(ois, Coder.Context.NESTED); + combineFn = (Combine.CombineFn) ois.readObject(); + inCoder = (Coder) ois.readObject(); + state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) + .decode(ois, Coder.Context.NESTED); } } /** * states correspond to dataflow objects. this one => serializable function */ - public static class SerFunctionState implements State { + public static class SerFunctionState implements State { - private final SerializableFunction, Out> sfunc; - private Out state; + private final SerializableFunction, OUT> sfunc; + private OUT state; - public SerFunctionState(SerializableFunction, Out> sfunc) { + public SerFunctionState(SerializableFunction, OUT> sfunc) { this.sfunc = sfunc; - this.state = sfunc.apply(ImmutableList.of()); + this.state = sfunc.apply(ImmutableList.of()); } @Override - public void update(In element) { + public void update(IN element) { @SuppressWarnings("unchecked") - In thisState = (In) state; + IN thisState = (IN) state; this.state = sfunc.apply(ImmutableList.of(element, thisState)); } @Override - public State merge(State other) { + public State merge(State other) { // Add exception catching and logging here. @SuppressWarnings("unchecked") - In thisState = (In) state; + IN thisState = (IN) state; @SuppressWarnings("unchecked") - In otherCurrent = (In) other.current(); + IN otherCurrent = (IN) other.current(); this.state = sfunc.apply(ImmutableList.of(thisState, otherCurrent)); return this; } @Override - public Out current() { + public OUT current() { return state; } @Override - public Out render() { + public OUT render() { return state; } } 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 index 8cb8bb3f116e..673244e4a10c 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java @@ -41,11 +41,11 @@ public class AvroPipelineTest { - private transient File inputFile; - private transient File outputDir; + private File inputFile; + private File outputDir; @Rule - public transient TemporaryFolder tmpDir = new TemporaryFolder(); + public final TemporaryFolder tmpDir = new TemporaryFolder(); @Before public void setUp() throws IOException { @@ -75,28 +75,26 @@ public void testGeneric() throws Exception { private void populateGenericFile(List genericRecords, Schema schema) throws IOException { FileOutputStream outputStream = new FileOutputStream(this.inputFile); - GenericDatumWriter genericDatumWriter = new GenericDatumWriter(schema); + GenericDatumWriter genericDatumWriter = new GenericDatumWriter<>(schema); - DataFileWriter dataFileWriter = new DataFileWriter(genericDatumWriter); - dataFileWriter.create(schema, outputStream); - - for (GenericRecord record : genericRecords) { - dataFileWriter.append(record); + try (DataFileWriter dataFileWriter = new DataFileWriter<>(genericDatumWriter)) { + dataFileWriter.create(schema, outputStream); + for (GenericRecord record : genericRecords) { + dataFileWriter.append(record); + } } - - dataFileWriter.close(); outputStream.close(); } private List readGenericFile() throws IOException { List records = Lists.newArrayList(); - GenericDatumReader genericDatumReader = new GenericDatumReader(); - DataFileReader dataFileReader = new DataFileReader - (new File(outputDir, "part-r-00000.avro"), genericDatumReader); - for (GenericRecord record : dataFileReader) { - records.add(record); + GenericDatumReader genericDatumReader = new GenericDatumReader<>(); + try (DataFileReader dataFileReader = new DataFileReader<> + (new File(outputDir, "part-r-00000.avro"), genericDatumReader)) { + for (GenericRecord record : dataFileReader) { + records.add(record); + } } - dataFileReader.close(); return records; } 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 index 5bbfe76f7b7f..f9d5b46f40c0 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombinePerKeyTest.java @@ -20,7 +20,11 @@ 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.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; @@ -33,8 +37,8 @@ public class CombinePerKeyTest { - private List WORDS = ImmutableList.of("the", "quick", "brown", "fox", "jumped", - "over", "the", "lazy", "dog"); + private static final List WORDS = + ImmutableList.of("the", "quick", "brown", "fox", "jumped", "over", "the", "lazy", "dog"); @Test public void testRun() { Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); @@ -51,6 +55,7 @@ public void testRun() { } private static class SumPerKey extends PTransform, PCollection>> { + @Override public PCollection> apply(PCollection pcol) { PCollection> withLongs = pcol.apply(ParDo.of(new DoFn>() { @Override 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 index 38dfd615fe12..bc45d8d4aedf 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -34,8 +34,8 @@ import java.io.FileFilter; import java.io.IOException; import java.net.URI; -import java.net.URISyntaxException; import java.util.HashSet; +import java.util.Locale; import java.util.Set; import org.junit.Before; import org.junit.Rule; @@ -52,11 +52,11 @@ */ public class TfIdfTest { - private transient File inputDir; - private transient File outputDir; + private File inputDir; + private File outputDir; @Rule - public transient TemporaryFolder tmpDir = new TemporaryFolder(); + public final TemporaryFolder tmpDir = new TemporaryFolder(); @Before public void setUp() throws IOException { @@ -67,7 +67,7 @@ public void setUp() throws IOException { outputDir.delete(); } - private void copy(String resourceName, File dir) throws IOException { + private static void copy(String resourceName, File dir) throws IOException { File dest = new File(dir, resourceName); Resources.copy(Resources.getResource(resourceName), Files.asByteSink(dest).openStream()); } @@ -101,8 +101,7 @@ public void testRun() throws Exception { fail("Could not find 'love' in output."); } - public Set listInputDocuments() - throws URISyntaxException, IOException { + public Set listInputDocuments() { Set uris = new HashSet<>(); File directory = inputDir; for (String entry : directory.list()) { @@ -118,9 +117,8 @@ public Set listInputDocuments() */ public static class ReadDocuments extends PTransform>> { - private static final long serialVersionUID = 0; - private Iterable uris; + private final Iterable uris; public ReadDocuments(Iterable uris) { this.uris = uris; @@ -143,9 +141,9 @@ public PCollection> apply(PInput input) { // TextIO.Read supports: // - file: URIs and paths locally // - gs: URIs on the service - for (final URI uri : uris) { + for (URI uri : uris) { String uriString; - if (uri.getScheme().equals("file")) { + if ("file".equals(uri.getScheme())) { uriString = new File(uri).getPath(); } else { uriString = uri.toString(); @@ -171,9 +169,6 @@ public PCollection> apply(PInput input) { */ public static class ComputeTfIdf extends PTransform>, PCollection>>> { - private static final long serialVersionUID = 0; - - public ComputeTfIdf() { } @Override public PCollection>> apply( @@ -194,20 +189,20 @@ public PCollection>> apply( PCollection> uriToWords = uriToContent .apply(ParDo.named("SplitWords").of( new DoFn, KV>() { - private static final long serialVersionUID = 0; - @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey(); String line = c.element().getValue(); for (String word : line.split("\\W+")) { - // Log INFO messages when the word “love” is found. - if (word.toLowerCase().equals("love")) { - LOG.info("Found {}", word.toLowerCase()); + if (LOG.isDebugEnabled()) { + // Log messages when the word "love" is found. + if ("love".equals(word.toLowerCase(Locale.ENGLISH))) { + LOG.debug("Found {}", word.toLowerCase(Locale.ENGLISH)); + } } if (!word.isEmpty()) { - c.output(KV.of(uri, word.toLowerCase())); + c.output(KV.of(uri, word.toLowerCase(Locale.ENGLISH))); } } } @@ -239,8 +234,6 @@ public void processElement(ProcessContext c) { PCollection>> uriToWordAndCount = uriAndWordToCount .apply(ParDo.named("ShiftKeys").of( new DoFn, Long>, KV>>() { - private static final long serialVersionUID = 0; - @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey().getKey(); @@ -256,8 +249,8 @@ public void processElement(ProcessContext c) { // a tuple tag. Each input must have the same key type, URI // in this case. The type parameter of the tuple tag matches // the types of the values for each collection. - final TupleTag wordTotalsTag = new TupleTag(); - final TupleTag> wordCountsTag = new TupleTag>(); + final TupleTag wordTotalsTag = new TupleTag<>(); + final TupleTag> wordCountsTag = new TupleTag<>(); KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple .of(wordTotalsTag, uriToWordTotal) .and(wordCountsTag, uriToWordAndCount); @@ -280,8 +273,6 @@ public void processElement(ProcessContext c) { PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal .apply(ParDo.named("ComputeTermFrequencies").of( new DoFn, KV>>() { - private static final long serialVersionUID = 0; - @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey(); @@ -308,8 +299,6 @@ public void processElement(ProcessContext c) { .named("ComputeDocFrequencies") .withSideInputs(totalDocuments) .of(new DoFn, KV>() { - private static final long serialVersionUID = 0; - @Override public void processElement(ProcessContext c) { String word = c.element().getKey(); @@ -324,8 +313,8 @@ public void processElement(ProcessContext c) { // Join the term frequency and document frequency // collections, each keyed on the word. - final TupleTag> tfTag = new TupleTag>(); - final TupleTag dfTag = new TupleTag(); + final TupleTag> tfTag = new TupleTag<>(); + final TupleTag dfTag = new TupleTag<>(); PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple .of(tfTag, wordToUriAndTf) .and(dfTag, wordToDf) @@ -336,11 +325,10 @@ public void processElement(ProcessContext c) { // ("term frequency - inverse document frequency") score; // here we use a basic version that is the term frequency // divided by the log of the document frequency. - PCollection>> wordToUriAndTfIdf = wordToUriAndTfAndDf + + return wordToUriAndTfAndDf .apply(ParDo.named("ComputeTfIdf").of( new DoFn, KV>>() { - private static final long serialVersionUID = 0; - @Override public void processElement(ProcessContext c) { String word = c.element().getKey(); @@ -354,8 +342,6 @@ public void processElement(ProcessContext c) { } } })); - - return wordToUriAndTfIdf; } // Instantiate Logger. @@ -370,9 +356,8 @@ public void processElement(ProcessContext c) { */ public static class WriteTfIdf extends PTransform>>, PDone> { - private static final long serialVersionUID = 0; - private String output; + private final String output; public WriteTfIdf(String output) { this.output = output; @@ -381,10 +366,7 @@ public WriteTfIdf(String output) { @Override public PDone apply(PCollection>> wordToUriAndTfIdf) { return wordToUriAndTfIdf - .apply(ParDo.named("Format").of(new DoFn>, String> - () { - private static final long serialVersionUID = 0; - + .apply(ParDo.named("Format").of(new DoFn>, String>() { @Override public void processElement(ProcessContext c) { c.output(String.format("%s,\t%s,\t%f", 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 index 3547d1a43565..5bddb526a4a0 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java @@ -15,6 +15,7 @@ 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; @@ -35,9 +36,6 @@ import java.nio.file.Paths; import java.util.List; -import static com.google.api.client.repackaged.com.google.common.base.Joiner.on; -import static java.io.File.separator; - /** * A test for the transforms registered in TransformTranslator. * Builds a regular Dataflow pipeline with each of the mapped @@ -46,7 +44,8 @@ */ public class TransformTranslatorTest { - @Rule public TestName name = new TestName(); + @Rule + public TestName name = new TestName(); private Pipeline testPipeline; private DirectPipelineRunner directRunner; @@ -59,7 +58,8 @@ public class TransformTranslatorTest { testPipeline = Pipeline.create(PipelineOptionsFactory.create()); sparkRunner = SparkPipelineRunner.create(); directRunner = DirectPipelineRunner.createForTest(); - testDataDirName = on(separator).join("target", "test-data", name.getMethodName()) + separator; + testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName()) + + File.separator; FileUtils.deleteDirectory(new File(testDataDirName)); new File(testDataDirName).mkdirs(); } @@ -74,19 +74,20 @@ public void run() { * 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 outFile = on(separator).join(testDataDirName, "test_text_out"); + @Test + public void testTextIOReadAndWriteTransforms() throws IOException { + String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out"); PCollection lines = testPipeline .apply(TextIO.Read.from("src/test/resources/test_text.txt")); lines.apply(TextIO.Write.to(outFile)); run(); - List directOutput = Files.readAllLines(Paths.get(outFile + "-00000-of-00001"), - Charsets.UTF_8); + List directOutput = + Files.readAllLines(Paths.get(outFile + "-00000-of-00001"), Charsets.UTF_8); - List sparkOutput = Files.readAllLines(Paths.get( - on(separator).join(outFile, "part-00000")), - Charsets.UTF_8); + List sparkOutput = + Files.readAllLines(Paths.get(Joiner.on(File.separator).join(outFile, "part-00000")), + Charsets.UTF_8); Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray()); } From 06e611a7516ef2fc61d43b1edd7aa6a7c3caf194 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 May 2015 14:51:27 +0100 Subject: [PATCH 054/137] Update to Spark 1.3. Closes issue #38 --- runners/spark/.travis.yml | 3 --- runners/spark/pom.xml | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/runners/spark/.travis.yml b/runners/spark/.travis.yml index 2eeae9d1e69d..6b91c9e4dacb 100644 --- a/runners/spark/.travis.yml +++ b/runners/spark/.travis.yml @@ -7,9 +7,6 @@ matrix: # Covers Java 7, Open JDK, and code coverage - jdk: openjdk7 env: JACOCO=-Pjacoco - # Covers Spark 1.3 - - jdk: openjdk7 - env: SPARK=-Dspark.version=1.3.1 # Covers Java 8, Oracle JDK - jdk: oraclejdk8 env: JAVA=-Djava.version=1.8 diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 926962482252..0b839070f38e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,7 +22,7 @@ License. UTF-8 UTF-8 1.7 - 1.2.2 + 1.3.1 com.google.cloud.dataflow.examples.WordCount /tmp/kinglear.txt /tmp/out From fe5a2bc8cfc16409b8768565fac16aac5d862928 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 May 2015 15:19:27 +0100 Subject: [PATCH 055/137] Add CONTRIBUTING.md to clarify contribution license; add build flair to README.md --- runners/spark/CONTRIBUTING.md | 8 ++++++++ runners/spark/README.md | 5 ++++- 2 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 runners/spark/CONTRIBUTING.md diff --git a/runners/spark/CONTRIBUTING.md b/runners/spark/CONTRIBUTING.md new file mode 100644 index 000000000000..1781a8662b3e --- /dev/null +++ b/runners/spark/CONTRIBUTING.md @@ -0,0 +1,8 @@ +## Licensing + +Contributions via GitHub pull requests are gladly accepted from their original author. +Along with any pull requests, please state that the contribution is your original work and +that you license the work to the project under the project's open source license. +Whether or not you state this explicitly, by submitting any copyrighted material via +pull request, email, or other means you agree to license the material under the project's +open source license and warrant that you have the legal authority to do so. diff --git a/runners/spark/README.md b/runners/spark/README.md index 7dceba6befbf..36a78ff131ed 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -74,4 +74,7 @@ Check the output by running: head /tmp/out/part-00000 -[wc]: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java \ No newline at end of file +[wc]: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java + +[![Build Status](https://travis-ci.org/cloudera/spark-dataflow.png?branch=master)](https://travis-ci.org/cloudera/spark-dataflow) +[![codecov.io](https://codecov.io/github/cloudera/spark-dataflow/coverage.svg?branch=master)](https://codecov.io/github/cloudera/spark-dataflow?branch=master) \ No newline at end of file From 2afc03b2910d8ac7f81d0ac95740ea1de1891666 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 May 2015 18:10:32 +0100 Subject: [PATCH 056/137] Enable RAT during verify phase and add some missing copyright headers --- runners/spark/build-resources/header-file.txt | 2 +- runners/spark/pom.xml | 13 +++++++++++++ .../com/cloudera/dataflow/spark/DeDupTest.java | 15 +++++++++++++++ .../com/cloudera/dataflow/spark/TfIdfTest.java | 15 +++++++++++++++ 4 files changed, 44 insertions(+), 1 deletion(-) diff --git a/runners/spark/build-resources/header-file.txt b/runners/spark/build-resources/header-file.txt index 19f441881819..6d81b4dc254a 100644 --- a/runners/spark/build-resources/header-file.txt +++ b/runners/spark/build-resources/header-file.txt @@ -1,5 +1,5 @@ /* - * Copyright (c) 2014, Cloudera, Inc. All Rights Reserved. + * 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 diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 0b839070f38e..239e3feadc5f 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -182,6 +182,15 @@ License. false + + + verify + verify + + check + + + org.codehaus.mojo @@ -226,6 +235,10 @@ License. org.apache.maven.plugins maven-checkstyle-plugin + + org.apache.rat + apache-rat-plugin + org.apache.maven.plugins maven-source-plugin 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 index 8eb6e55b647e..7179c166aad2 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java @@ -1,3 +1,18 @@ +/* + * 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; 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 index bc45d8d4aedf..6abd33c3e31d 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -1,3 +1,18 @@ +/* + * 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; From 64ab065ffc4b60b09e6fe2b32680dd540a7a06a8 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 15 May 2015 09:47:14 +0100 Subject: [PATCH 057/137] Remove unnecessary build properties for mvn exec. --- runners/spark/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 239e3feadc5f..5d22e8bb4b98 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,11 +23,6 @@ License. UTF-8 1.7 1.3.1 - com.google.cloud.dataflow.examples.WordCount - /tmp/kinglear.txt - /tmp/out - SparkPipelineRunner - local From cdb96658c404d32d08fd7741c2bce5de3e59f706 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 15 May 2015 10:44:06 +0100 Subject: [PATCH 058/137] Document how to run on a cluster with spark-submit. This change creates a Spark application JAR that can be used to run the word count example on the cluster. The Spark depdendency is now marked 'provided' to make it possible to run on different (minor) versions of Spark. Note also that Guava is shaded in the application JAR to avoid conflicts with the version used by Hadoop. --- runners/spark/README.md | 24 ++++++++++++++++++ runners/spark/pom.xml | 54 +++++++++++++++++++++++++++++++++++------ 2 files changed, 70 insertions(+), 8 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 36a78ff131ed..54832f364beb 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -74,7 +74,31 @@ Check the output by running: head /tmp/out/part-00000 +__Note: running examples using `mvn exec:exec` only works for Spark local mode at the +moment. See the next section for how to run on a cluster.__ + [wc]: https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +## Running on a Cluster + +Spark Dataflow pipelines can be run on a cluster using the `spark-submit` command. + +First copy a text document to HDFS: + + curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt | hadoop fs -put - kinglear.txt + +Then run the word count example using Spark submit with the `yarn-client` master +(`yarn-cluster` works just as well): + + spark-submit \ + --class com.google.cloud.dataflow.examples.WordCount \ + --master yarn-client \ + target/spark-dataflow-*-spark-app.jar \ + --input=kinglear.txt --output=out --runner=SparkPipelineRunner --sparkMaster=yarn-client + +Check the output by running: + + hadoop fs -tail out/part-00000 + [![Build Status](https://travis-ci.org/cloudera/spark-dataflow.png?branch=master)](https://travis-ci.org/cloudera/spark-dataflow) [![codecov.io](https://codecov.io/github/cloudera/spark-dataflow/coverage.svg?branch=master)](https://codecov.io/github/cloudera/spark-dataflow?branch=master) \ No newline at end of file diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 5d22e8bb4b98..c10f4e91d0d4 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -223,6 +223,33 @@ License. + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + package + + shade + + + + + + com.google.common + com.cloudera.dataflow.spark.relocated.com.google.common + + + true + spark-app + + + + + + + @@ -242,6 +269,10 @@ License. org.apache.maven.plugins maven-javadoc-plugin + + org.apache.maven.plugins + maven-shade-plugin + @@ -250,14 +281,7 @@ License. org.apache.spark spark-core_2.10 ${spark.version} - - - - org.slf4j - jul-to-slf4j - - + provided com.google.guava @@ -268,11 +292,25 @@ License. com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all 0.4.150414 + + + + org.slf4j + slf4j-jdk14 + + com.google.cloud.dataflow google-cloud-dataflow-java-examples-all 0.4.150414 + + + + org.slf4j + slf4j-jdk14 + + org.apache.avro From b8190ea7e78452de6f855357efda651fe78c50e2 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 12 May 2015 08:36:37 +0100 Subject: [PATCH 059/137] Support withNumShards() and withoutSharding() for TextIO output Fixes #26 --- .../dataflow/spark/TransformTranslator.java | 13 +++ .../dataflow/spark/NumShardsTest.java | 85 +++++++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.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 index 4030395d8a9c..074a76a3a0c4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -294,6 +294,19 @@ public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) @SuppressWarnings("unchecked") JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); String pattern = transform.getFilenamePrefix(); + if (transform.getNumShards() > 0) { + last = last.mapToPair(new PairFunction() { + @Override + public Tuple2 call(T t) throws Exception { + return new Tuple2<>(t, null); + }}) + .repartition(transform.getNumShards()) + .map(new Function, T>() { + @Override + public T call(Tuple2 tuple) throws Exception { + return tuple._1(); + }}); + } last.saveAsTextFile(pattern); } }; 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 new file mode 100644 index 000000000000..db1e6863471b --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java @@ -0,0 +1,85 @@ +/* + * 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.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 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 inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + PCollection output = inputWords.apply(new WordCount.CountWords()); + output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3)); + p.run(); + + int count = 0; + Set expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2"); + for (File f : outputDir.listFiles(new FileFilter() { + @Override public boolean accept(File pathname) { + return pathname.getName().startsWith("part-"); + } + })) { + count++; + for (String line : Files.readLines(f, Charsets.UTF_8)) { + assertTrue(line + " not found", expected.remove(line)); + } + } + assertEquals(3, count); + assertTrue(expected.isEmpty()); + } + +} From d82355cac658ff861aa301b48b6fb937698eedfd Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 15 May 2015 17:43:09 +0100 Subject: [PATCH 060/137] Implement dataflow sharding for text output. With this change the direct runner and the Spark runner both produce output files with the same names. Fixes #36 --- .../dataflow/spark/ShardNameBuilder.java | 86 +++++++++++++++++++ .../spark/TemplatedTextOutputFormat.java | 51 +++++++++++ .../dataflow/spark/TransformTranslator.java | 37 +++++--- .../dataflow/spark/NumShardsTest.java | 6 +- .../dataflow/spark/ShardNameBuilderTest.java | 68 +++++++++++++++ .../cloudera/dataflow/spark/TfIdfTest.java | 7 +- .../spark/TransformTranslatorTest.java | 32 +++---- 7 files changed, 251 insertions(+), 36 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.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 new file mode 100644 index 000000000000..f838cc274d7a --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java @@ -0,0 +1,86 @@ +/* + * 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 + * @param suffix a filename suffix + * @return the output directory for the given prefix, template and suffix + */ + public static String getOutputDirectory(String pathPrefix, String template, + String suffix) { + return new Path(pathPrefix + template + suffix).getParent().toString(); + } + + /** + * @param pathPrefix a relative or absolute path + * @param template a template string + * @param suffix a filename suffix + * @return the output filename for the given prefix, template and suffix + */ + public static String getOutputFile(String pathPrefix, String template, + String suffix) { + return new Path(pathPrefix + template + suffix).getName(); + } +} 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 new file mode 100644 index 000000000000..6783b0726d3e --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java @@ -0,0 +1,51 @@ +/* + * 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.TaskID; +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; + +public class TemplatedTextOutputFormat extends TextOutputFormat { + + public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.textoutputformat.template"; + + @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 + FileOutputCommitter committer = + (FileOutputCommitter) getOutputCommitter(context); + return new Path(committer.getWorkPath(), getOutputFile(context)); + } + + private String getOutputFile(TaskAttemptContext context) { + TaskID taskId = context.getTaskAttemptID().getTaskID(); + int partition = taskId.getId(); + String outputFileTemplate = context.getConfiguration().get(OUTPUT_FILE_TEMPLATE); + return ShardNameBuilder.replaceShardNumber(outputFileTemplate, partition); + } +} 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 index 074a76a3a0c4..0342b51b9ca6 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -49,6 +49,7 @@ import org.apache.avro.mapreduce.AvroKeyOutputFormat; 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.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -292,22 +293,34 @@ private static TransformEvaluator> writeText() { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { @SuppressWarnings("unchecked") - JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); - String pattern = transform.getFilenamePrefix(); - if (transform.getNumShards() > 0) { - last = last.mapToPair(new PairFunction() { + JavaPairRDD last = ((JavaRDDLike) context.getInputRDD(transform)) + .mapToPair(new PairFunction() { @Override public Tuple2 call(T t) throws Exception { return new Tuple2<>(t, null); - }}) - .repartition(transform.getNumShards()) - .map(new Function, T>() { - @Override - public T call(Tuple2 tuple) throws Exception { - return tuple._1(); - }}); + } + }); + int shardCount = transform.getNumShards(); + if (shardCount == 0) { + // use default number of shards, but find the actual number for the template + shardCount = last.partitions().size(); + } else { + // number of shards was set explicitly, so repartition + last = last.repartition(transform.getNumShards()); } - last.saveAsTextFile(pattern); + + String template = ShardNameBuilder.replaceShardCount(transform.getShardTemplate(), + shardCount); + String outputDir = ShardNameBuilder.getOutputDirectory(transform.getFilenamePrefix(), + template, transform.getFilenameSuffix()); + String fileTemplate = ShardNameBuilder.getOutputFile(transform.getFilenamePrefix(), + template, transform.getFilenameSuffix()); + + Configuration conf = new Configuration(); + conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_TEMPLATE, fileTemplate); + last.saveAsNewAPIHadoopFile(outputDir, Text.class, NullWritable.class, + TemplatedTextOutputFormat.class, conf); } }; } 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 index db1e6863471b..ea6fcfc33a93 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java @@ -63,14 +63,14 @@ public void testText() throws Exception { Pipeline p = Pipeline.create(options); PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(new WordCount.CountWords()); - output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3)); + output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt")); p.run(); int count = 0; Set expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2"); - for (File f : outputDir.listFiles(new FileFilter() { + for (File f : tmpDir.getRoot().listFiles(new FileFilter() { @Override public boolean accept(File pathname) { - return pathname.getName().startsWith("part-"); + return pathname.getName().matches("out-.*\\.txt"); } })) { count++; 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 new file mode 100644 index 000000000000..dc48b51be257 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java @@ -0,0 +1,68 @@ +/* + * 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.getOutputFile; +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("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", ".txt")); + 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", ".txt")); + } + + @Test + public void testGetOutputFile() { + assertEquals("bar-S-of-N", getOutputFile("foo/bar", "-S-of-N", "")); + assertEquals("bar-S-of-N", getOutputFile("/foo/bar", "-S-of-N", "")); + assertEquals("bar-S-of-N.txt", getOutputFile("hdfs://foo/bar", "-S-of-N", ".txt")); + assertEquals("part-SSSSS", getOutputFile("foo/bar", "/part-SSSSS", "")); + assertEquals("part-SSSSS", getOutputFile("/foo/bar", "/part-SSSSS", "")); + assertEquals("part-SSSSS.txt", getOutputFile("hdfs://foo/bar", "/part-SSSSS", ".txt")); + } + +} 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 index 6abd33c3e31d..48620607fc73 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -100,9 +100,10 @@ public void testRun() throws Exception { SparkPipelineRunner.create().run(pipeline); - for (File f : outputDir.listFiles(new FileFilter() { - @Override public boolean accept(File pathname) { - return pathname.getName().startsWith("part-"); + for (File f : tmpDir.getRoot().listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return pathname.getName().startsWith("out-"); } })) { for (String line : Files.readLines(f, Charsets.UTF_8)) { 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 index 5bddb526a4a0..0251808b441b 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java @@ -20,7 +20,7 @@ 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.DirectPipelineRunner.EvaluationResults; +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; @@ -47,15 +47,11 @@ public class TransformTranslatorTest { @Rule public TestName name = new TestName(); - private Pipeline testPipeline; private DirectPipelineRunner directRunner; private SparkPipelineRunner sparkRunner; - private EvaluationResults directRunResult; - private EvaluationResult sparkRunResult; private String testDataDirName; @Before public void init() throws IOException { - testPipeline = Pipeline.create(PipelineOptionsFactory.create()); sparkRunner = SparkPipelineRunner.create(); directRunner = DirectPipelineRunner.createForTest(); testDataDirName = Joiner.on(File.separator).join("target", "test-data", name.getMethodName()) @@ -64,11 +60,6 @@ public class TransformTranslatorTest { new File(testDataDirName).mkdirs(); } - public void run() { - directRunResult = directRunner.run(testPipeline); - sparkRunResult = sparkRunner.run(testPipeline); - } - /** * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark @@ -76,19 +67,24 @@ public void run() { */ @Test public void testTextIOReadAndWriteTransforms() throws IOException { - String outFile = Joiner.on(File.separator).join(testDataDirName, "test_text_out"); - PCollection lines = testPipeline - .apply(TextIO.Read.from("src/test/resources/test_text.txt")); - lines.apply(TextIO.Write.to(outFile)); - run(); + String directOut = runPipeline("direct", directRunner); + String sparkOut = runPipeline("spark", sparkRunner); List directOutput = - Files.readAllLines(Paths.get(outFile + "-00000-of-00001"), Charsets.UTF_8); + Files.readAllLines(Paths.get(directOut + "-00000-of-00001"), Charsets.UTF_8); List sparkOutput = - Files.readAllLines(Paths.get(Joiner.on(File.separator).join(outFile, "part-00000")), - Charsets.UTF_8); + Files.readAllLines(Paths.get(sparkOut + "-00000-of-00001"), Charsets.UTF_8); 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 lines = p.apply(TextIO.Read.from("src/test/resources/test_text.txt")); + lines.apply(TextIO.Write.to(outFile)); + runner.run(p); + return outFile; + } } From 1df47a1e74699b863def4d3eee8758ae97c6af7c Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 19 May 2015 09:35:17 +0100 Subject: [PATCH 061/137] Fix README to reflect changes from b6e4787. Also fixes a corner case where the output directory is the current (home) directory. --- runners/spark/README.md | 4 ++-- runners/spark/pom.xml | 1 + .../java/com/cloudera/dataflow/spark/ShardNameBuilder.java | 6 +++++- .../com/cloudera/dataflow/spark/ShardNameBuilderTest.java | 2 ++ 4 files changed, 10 insertions(+), 3 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 54832f364beb..6ca5b3da01ed 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -72,7 +72,7 @@ in local mode: Check the output by running: - head /tmp/out/part-00000 + head /tmp/out-00000-of-00001 __Note: running examples using `mvn exec:exec` only works for Spark local mode at the moment. See the next section for how to run on a cluster.__ @@ -98,7 +98,7 @@ Then run the word count example using Spark submit with the `yarn-client` master Check the output by running: - hadoop fs -tail out/part-00000 + hadoop fs -tail out-00000-of-00002 [![Build Status](https://travis-ci.org/cloudera/spark-dataflow.png?branch=master)](https://travis-ci.org/cloudera/spark-dataflow) [![codecov.io](https://codecov.io/github/cloudera/spark-dataflow/coverage.svg?branch=master)](https://codecov.io/github/cloudera/spark-dataflow?branch=master) \ No newline at end of file diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index c10f4e91d0d4..11421de0c215 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -193,6 +193,7 @@ License. 1.4.0 java + test -classpath 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 index f838cc274d7a..9cc8c36cda78 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java @@ -70,7 +70,11 @@ private static String replaceShardPattern(String template, String pattern, int n */ public static String getOutputDirectory(String pathPrefix, String template, String suffix) { - return new Path(pathPrefix + template + suffix).getParent().toString(); + String out = new Path(pathPrefix + template + suffix).getParent().toString(); + if (out.isEmpty()) { + return "./"; + } + return out; } /** 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 index dc48b51be257..178611553e79 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java @@ -47,6 +47,7 @@ public void testReplaceShardNumber() { @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", ".txt")); @@ -57,6 +58,7 @@ public void testGetOutputDirectory() { @Test public void testGetOutputFile() { + assertEquals("foo-S-of-N", getOutputFile("foo", "-S-of-N", "")); assertEquals("bar-S-of-N", getOutputFile("foo/bar", "-S-of-N", "")); assertEquals("bar-S-of-N", getOutputFile("/foo/bar", "-S-of-N", "")); assertEquals("bar-S-of-N.txt", getOutputFile("hdfs://foo/bar", "-S-of-N", ".txt")); From ea2c4ad1b168ac824e4f007369d9f33fd02f6fb0 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 19 May 2015 12:35:22 +0100 Subject: [PATCH 062/137] Shard number replacement should only be for 'S' pattern in the template, not anywhere in the filename. --- .../dataflow/spark/ShardNameBuilder.java | 34 ++++++++++---- .../spark/TemplatedTextOutputFormat.java | 11 ++++- .../dataflow/spark/TransformTranslator.java | 18 +++++--- .../dataflow/spark/ShardNameBuilderTest.java | 44 ++++++++++++------- 4 files changed, 74 insertions(+), 33 deletions(-) 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 index 9cc8c36cda78..f53b6d906e55 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameBuilder.java @@ -65,12 +65,10 @@ private static String replaceShardPattern(String template, String pattern, int n /** * @param pathPrefix a relative or absolute path * @param template a template string - * @param suffix a filename suffix * @return the output directory for the given prefix, template and suffix */ - public static String getOutputDirectory(String pathPrefix, String template, - String suffix) { - String out = new Path(pathPrefix + template + suffix).getParent().toString(); + public static String getOutputDirectory(String pathPrefix, String template) { + String out = new Path(pathPrefix + template).getParent().toString(); if (out.isEmpty()) { return "./"; } @@ -80,11 +78,29 @@ public static String getOutputDirectory(String pathPrefix, String template, /** * @param pathPrefix a relative or absolute path * @param template a template string - * @param suffix a filename suffix - * @return the output filename for the given prefix, template and suffix + * @return the prefix of the output filename for the given path prefix and template */ - public static String getOutputFile(String pathPrefix, String template, - String suffix) { - return new Path(pathPrefix + template + suffix).getName(); + 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; + } } } 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 index 6783b0726d3e..5d00900aeac4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java @@ -24,9 +24,13 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardNumber; + public class TemplatedTextOutputFormat extends TextOutputFormat { + public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.textoutputformat.prefix"; public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.textoutputformat.template"; + public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.textoutputformat.suffix"; @Override public void checkOutputSpecs(JobContext job) { @@ -45,7 +49,10 @@ public Path getDefaultWorkFile(TaskAttemptContext context, private String getOutputFile(TaskAttemptContext context) { TaskID taskId = context.getTaskAttemptID().getTaskID(); int partition = taskId.getId(); - String outputFileTemplate = context.getConfiguration().get(OUTPUT_FILE_TEMPLATE); - return ShardNameBuilder.replaceShardNumber(outputFileTemplate, partition); + + 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; } } 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 index 0342b51b9ca6..ab463dc898fc 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -60,6 +60,11 @@ 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; + /** * Supports translation between a DataFlow transform, and Spark's operations on RDDs. */ @@ -310,15 +315,16 @@ public Tuple2 call(T t) throws Exception { last = last.repartition(transform.getNumShards()); } - String template = ShardNameBuilder.replaceShardCount(transform.getShardTemplate(), - shardCount); - String outputDir = ShardNameBuilder.getOutputDirectory(transform.getFilenamePrefix(), - template, transform.getFilenameSuffix()); - String fileTemplate = ShardNameBuilder.getOutputFile(transform.getFilenamePrefix(), - template, transform.getFilenameSuffix()); + String template = replaceShardCount(transform.getShardTemplate(), shardCount); + String outputDir = getOutputDirectory(transform.getFilenamePrefix(), template); + String filePrefix = getOutputFilePrefix(transform.getFilenamePrefix(), template); + String fileTemplate = getOutputFileTemplate(transform.getFilenamePrefix(), template); + String fileSuffix = transform.getFilenameSuffix(); Configuration conf = new Configuration(); + conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_PREFIX, filePrefix); conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_TEMPLATE, fileTemplate); + conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_SUFFIX, fileSuffix); last.saveAsNewAPIHadoopFile(outputDir, Text.class, NullWritable.class, TemplatedTextOutputFormat.class, conf); } 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 index 178611553e79..341c21423562 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/ShardNameBuilderTest.java @@ -18,7 +18,8 @@ import org.junit.Test; import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputDirectory; -import static com.cloudera.dataflow.spark.ShardNameBuilder.getOutputFile; +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; @@ -47,24 +48,35 @@ public void testReplaceShardNumber() { @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", ".txt")); - 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", ".txt")); + 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 testGetOutputFile() { - assertEquals("foo-S-of-N", getOutputFile("foo", "-S-of-N", "")); - assertEquals("bar-S-of-N", getOutputFile("foo/bar", "-S-of-N", "")); - assertEquals("bar-S-of-N", getOutputFile("/foo/bar", "-S-of-N", "")); - assertEquals("bar-S-of-N.txt", getOutputFile("hdfs://foo/bar", "-S-of-N", ".txt")); - assertEquals("part-SSSSS", getOutputFile("foo/bar", "/part-SSSSS", "")); - assertEquals("part-SSSSS", getOutputFile("/foo/bar", "/part-SSSSS", "")); - assertEquals("part-SSSSS.txt", getOutputFile("hdfs://foo/bar", "/part-SSSSS", ".txt")); + 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")); } } From 5fb371263f11ccd5d8466b6d54035f40fa735428 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 19 May 2015 20:05:25 +0100 Subject: [PATCH 063/137] Use internal coder for side inputs, rather than trying to infer our own manually. (The latter doesn't work for empty collection types, like List.) This includes a test (EmptyInputTest) that fails without the main code change. --- .../cloudera/dataflow/spark/DoFnFunction.java | 7 ++- .../dataflow/spark/EvaluationContext.java | 17 ----- .../dataflow/spark/MultiDoFnFunction.java | 7 ++- .../dataflow/spark/TransformTranslator.java | 6 +- .../dataflow/spark/EmptyInputTest.java | 63 +++++++++++++++++++ 5 files changed, 76 insertions(+), 24 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.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 index b02b87ad99ec..0ea2b2b289a9 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -26,6 +26,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -92,9 +93,11 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { + BroadcastHelper broadcastHelper = mSideInputs.get(view.getTagInternal()); @SuppressWarnings("unchecked") - T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); - return value; + Iterable> contents = + (Iterable>) broadcastHelper.getValue(); + return view.fromIterableInternal(contents); } @Override 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 index 24ecdeb2aeda..1e71f5d980f6 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -24,7 +24,6 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; -import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -69,22 +68,6 @@ SparkRuntimeContext getRuntimeContext() { return runtime; } - Coder getDefaultCoder(T example) { - Coder defaultCoder = registry.getDefaultCoder(example); - if (defaultCoder == null) { - if (example instanceof Iterable) { - Object first = ((Iterable) example).iterator().next(); - @SuppressWarnings("unchecked") - Coder coder = (Coder) IterableCoder.of(getDefaultCoder(first)); - return coder; - } else { - throw new IllegalStateException(String.format("Couldn't determine the default coder for " + - "an example of class [%s]", example.getClass())); - } - } - return defaultCoder; - } - I getInput(PTransform transform) { @SuppressWarnings("unchecked") I input = (I) pipeline.getInput(transform); 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 index 37ddffb7ae7e..34966ccfa9be 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -97,9 +98,11 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { + BroadcastHelper broadcastHelper = mSideInputs.get(view.getTagInternal()); @SuppressWarnings("unchecked") - T value = (T) mSideInputs.get(view.getTagInternal()).getValue(); - return value; + Iterable> contents = + (Iterable>) broadcastHelper.getValue(); + return view.fromIterableInternal(contents); } @Override 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 index ab463dc898fc..50728b2df4c7 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -473,9 +473,9 @@ private static Map, BroadcastHelper> getSideInputs( } else { Map, BroadcastHelper> sideInputs = Maps.newHashMap(); for (PCollectionView view : views) { - Object sideinput = view.fromIterableInternal(context.getPCollectionView(view)); - Coder coder = context.getDefaultCoder(sideinput); - BroadcastHelper helper = new BroadcastHelper<>(sideinput, coder); + Iterable> collectionView = context.getPCollectionView(view); + Coder>> coderInternal = view.getCoderInternal(); + BroadcastHelper helper = new BroadcastHelper<>(collectionView, coderInternal); //broadcast side inputs helper.broadcast(context.getSparkContext()); sideInputs.put(view.getTagInternal(), helper); 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 new file mode 100644 index 000000000000..066af84e5289 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java @@ -0,0 +1,63 @@ +/* + * 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 empty = Collections.emptyList(); + PCollection inputWords = p.apply(Create.of(empty)).setCoder(StringUtf8Coder.of()); + PCollection output = inputWords.apply(Combine.globally(new ConcatWords())); + + EvaluationResult run = SparkPipelineRunner.create(options).run(p); + assertEquals("", Iterables.getOnlyElement(run.get(output))); + } + + public static class ConcatWords implements SerializableFunction, String> { + @Override + public String apply(Iterable 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(); + } + } + +} From 94a8f294176bed869cf4c88ebdae11f4bb482df2 Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 20 May 2015 11:30:02 +0100 Subject: [PATCH 064/137] Address review feedback. --- .../main/java/com/cloudera/dataflow/spark/DoFnFunction.java | 6 +++--- .../java/com/cloudera/dataflow/spark/MultiDoFnFunction.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) 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 index 0ea2b2b289a9..875297c81c3d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -93,10 +93,10 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - BroadcastHelper broadcastHelper = mSideInputs.get(view.getTagInternal()); @SuppressWarnings("unchecked") - Iterable> contents = - (Iterable>) broadcastHelper.getValue(); + BroadcastHelper>> broadcastHelper = + (BroadcastHelper>>) mSideInputs.get(view.getTagInternal()); + Iterable> contents = broadcastHelper.getValue(); return view.fromIterableInternal(contents); } 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 index 34966ccfa9be..38bccc428964 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -98,10 +98,10 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - BroadcastHelper broadcastHelper = mSideInputs.get(view.getTagInternal()); @SuppressWarnings("unchecked") - Iterable> contents = - (Iterable>) broadcastHelper.getValue(); + BroadcastHelper>> broadcastHelper = + (BroadcastHelper>>) mSideInputs.get(view.getTagInternal()); + Iterable> contents = broadcastHelper.getValue(); return view.fromIterableInternal(contents); } From 916e0393e7301d6682d1f1d5a6589231d7892cb2 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 21 May 2015 12:20:39 +0100 Subject: [PATCH 065/137] Use Coders to convert from object-based RDDs to byte-array RDDS for all operations that go through the shuffle. --- .../cloudera/dataflow/spark/ByteArray.java | 52 ++++++ .../cloudera/dataflow/spark/CoderHelpers.java | 69 +++++++ .../dataflow/spark/SparkPipelineRunner.java | 2 - .../dataflow/spark/TransformTranslator.java | 79 +++++--- .../dataflow/spark/SerializationTest.java | 173 ++++++++++++++++++ 5 files changed, 349 insertions(+), 26 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.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 new file mode 100644 index 000000000000..1db0a8bac5a4 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/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 com.cloudera.dataflow.spark; + +import java.io.Serializable; +import java.util.Arrays; + +import com.google.common.primitives.UnsignedBytes; + +class ByteArray implements Serializable, Comparable { + + 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); + } +} 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 index 4bf81e57cd10..b5e86b6f2198 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java @@ -22,7 +22,10 @@ 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. @@ -113,4 +116,70 @@ public T call(byte[] bytes) throws Exception { } }; } + + /** + * 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 The type of the key being serialized. + * @param The type of the value being serialized. + * @return A function that accepts a key-value pair and returns a pair of byte arrays. + */ + static PairFunction, ByteArray, byte[]> toByteFunction( + final Coder keyCoder, final Coder valueCoder) { + return new PairFunction, ByteArray, byte[]>() { + @Override + public Tuple2 call(Tuple2 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 The type of the key being deserialized. + * @param The type of the value being deserialized. + * @return A function that accepts a pair of byte arrays and returns a key-value pair. + */ + static PairFunction, K, V> fromByteFunction( + final Coder keyCoder, final Coder valueCoder) { + return new PairFunction, K, V>() { + @Override + public Tuple2 call(Tuple2 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 Iterable. + * + * @param keyCoder Coder to deserialize keys. + * @param valueCoder Coder to deserialize values. + * @param The type of the key being deserialized. + * @param The type of the value being deserialized. + * @return A function that accepts a pair of byte arrays and returns a key-value pair. + */ + static PairFunction>, K, Iterable> + fromByteFunctionIterable(final Coder keyCoder, final Coder valueCoder) { + return new PairFunction>, K, Iterable>() { + @Override + public Tuple2> call(Tuple2> tuple) { + return new Tuple2<>(fromByteArray(tuple._1().getValue(), keyCoder), + Iterables.transform(tuple._2(), new com.google.common.base.Function() { + @Override + public V apply(byte[] bytes) { + return fromByteArray(bytes, valueCoder); + } + })); + } + }; + } } 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 index b2e5150f2be8..6fdb5d7d2a8e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -24,7 +24,6 @@ import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.serializer.KryoSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,7 +111,6 @@ private JavaSparkContext getContext() { conf.setMaster(mOptions.getSparkMaster()); conf.setAppName("spark pipeline job"); conf.set("sun.io.serialization.extendeddebuginfo", "true"); - conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); return new JavaSparkContext(conf); } 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 index 50728b2df4c7..dbaf4d945803 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -22,6 +22,7 @@ import com.google.api.client.util.Maps; 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; @@ -118,7 +119,16 @@ public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContex @SuppressWarnings("unchecked") JavaRDDLike, ?> inRDD = (JavaRDDLike, ?>) context.getInputRDD(transform); - context.setOutputRDD(transform, fromPair(toPair(inRDD).groupByKey())); + @SuppressWarnings("unchecked") + KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); + final Coder keyCoder = coder.getKeyCoder(); + final Coder valueCoder = coder.getValueCoder(); + + JavaRDDLike>, ?> outRDD = fromPair(toPair(inRDD) + .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder)) + .groupByKey() + .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder))); + context.setOutputRDD(transform, outRDD); } }; } @@ -150,6 +160,15 @@ public void evaluate(Combine.PerKey transform, EvaluationContext cont JavaRDDLike, ?> inRdd = (JavaRDDLike, ?>) context.getInputRDD(transform); + @SuppressWarnings("unchecked") + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + Coder keyCoder = inputCoder.getKeyCoder(); + Coder viCoder = inputCoder.getValueCoder(); + Coder vaCoder = keyed.getAccumulatorCoder( + context.getPipeline().getCoderRegistry(), keyCoder, viCoder); + final Coder> kviCoder = KvCoder.of(keyCoder, viCoder); + final Coder> 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 @@ -164,42 +183,54 @@ public Tuple2> call(KV kv) { } }); + JavaPairRDD inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair + .mapToPair(CoderHelpers.toByteFunction(keyCoder, kviCoder)); + // 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> accumulated = inRddDuplicatedKeyPair.combineByKey( - new Function, KV>() { + JavaPairRDD*/ byte[]> accumulatedBytes = + inRddDuplicatedKeyPairBytes.combineByKey( + new Function*/ byte[], /*KV*/ byte[]>() { @Override - public KV call(KV input) { - VA acc = keyed.createAccumulator(input.getKey()); - keyed.addInput(input.getKey(), acc, input.getValue()); - return KV.of(input.getKey(), acc); + public /*KV*/ byte[] call(/*KV*/ byte[] input) { + KV kvi = CoderHelpers.fromByteArray(input, kviCoder); + VA va = keyed.createAccumulator(kvi.getKey()); + keyed.addInput(kvi.getKey(), va, kvi.getValue()); + return CoderHelpers.toByteArray(KV.of(kvi.getKey(), va), kvaCoder); } }, - new Function2, KV, KV>() { + new Function2*/ byte[], /*KV*/ byte[], /*KV*/ byte[]>() { @Override - public KV call(KV acc, KV input) { - keyed.addInput(acc.getKey(), acc.getValue(), input.getValue()); - return acc; + public /*KV*/ byte[] call(/*KV*/ byte[] acc, + /*KV*/ byte[] input) { + KV kva = CoderHelpers.fromByteArray(acc, kvaCoder); + KV kvi = CoderHelpers.fromByteArray(input, kviCoder); + keyed.addInput(kva.getKey(), kva.getValue(), kvi.getValue()); + return CoderHelpers.toByteArray(KV.of(kva.getKey(), kva.getValue()), kvaCoder); } }, - new Function2, KV, KV>() { + new Function2*/ byte[], /*KV*/ byte[], /*KV*/ byte[]>() { @Override - public KV call(KV acc1, KV acc2) { - return KV.of( - acc1.getKey(), - keyed.mergeAccumulators( - acc1.getKey(), ImmutableList.of(acc1.getValue(), acc2.getValue()))); + public /*KV*/ byte[] call(/*KV*/ byte[] acc1, + /*KV*/ byte[] acc2) { + KV kva1 = CoderHelpers.fromByteArray(acc1, kvaCoder); + KV kva2 = CoderHelpers.fromByteArray(acc2, kvaCoder); + VA va = keyed.mergeAccumulators(kva1.getKey(), + ImmutableList.of(kva1.getValue(), kva2.getValue())); + return CoderHelpers.toByteArray(KV.of(kva1.getKey(), va), kvaCoder); } }); - JavaPairRDD extracted = accumulated.mapValues( - new Function, VO>() { - @Override - public VO call(KV acc) { - return keyed.extractOutput(acc.getKey(), acc.getValue()); - } - }); + JavaPairRDD extracted = accumulatedBytes + .mapToPair(CoderHelpers.fromByteFunction(keyCoder, kvaCoder)) + .mapValues( + new Function, VO>() { + @Override + public VO call(KV acc) { + return keyed.extractOutput(acc.getKey(), acc.getValue()); + } + }); context.setOutputRDD(transform, fromPair(extracted)); } }; 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 new file mode 100644 index 000000000000..dff4bbb4da8a --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -0,0 +1,173 @@ +/* + * 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 org.junit.Test; + +public class SerializationTest { + + public static class StringHolder { // not serializable + private String string; + public StringHolder(String string) { + this.string = string; + } + @Override + public String toString() { + return string; + } + } + + public static class StringHolderUtf8Coder extends AtomicCoder { + + private 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)); + } + + @Override + public boolean isDeterministic() { + return true; + } + + public static Coder of() { + return new StringHolderUtf8Coder(); + } + } + + private static final String[] WORDS_ARRAY = { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + private static final List WORDS = Lists.transform( + Arrays.asList(WORDS_ARRAY), new Function() { + @Override public StringHolder apply(String s) { + return new StringHolder(s); + } + }); + private static final Set EXPECTED_COUNT_SET = + ImmutableSet.copyOf(Lists.transform( + Arrays.asList("hi: 5", "there: 1", "sue: 2", "bob: 2"), + new Function() { + @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 inputWords = p.apply(Create.of(WORDS)).setCoder + (StringHolderUtf8Coder.of()); + PCollection output = inputWords.apply(new CountWords()); + + DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + p.run(); + + } + + /** + * A DoFn that tokenizes lines of text into individual words. + */ + static class ExtractWordsFn extends DoFn { + private Aggregator emptyLines; + + @Override + public void startBundle(Context c) { + emptyLines = c.createAggregator("emptyLines", new Sum.SumLongFn()); + } + + @Override + public void processElement(ProcessContext c) { + // Split the line into words. + String[] words = c.element().toString().split("[^a-zA-Z']+"); + + // 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, StringHolder> { + @Override + public void processElement(ProcessContext c) { + c.output(new StringHolder(c.element().getKey() + ": " + c.element().getValue())); + } + } + + private static class CountWords extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + PCollection> wordCounts = + words.apply(Count.perElement()); + + // Format each word and count into a printable string. + + return wordCounts.apply(ParDo.of(new FormatCountsFn())); + } + + } +} From ec2f9e7bb95096078f5caf2d493d382f976ed4de Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 21 May 2015 21:42:24 +0100 Subject: [PATCH 066/137] Switch to use Kryo serializer. --- .../java/com/cloudera/dataflow/spark/SparkPipelineRunner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 index 6fdb5d7d2a8e..daa30f55bf58 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.serializer.KryoSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,7 +111,7 @@ private JavaSparkContext getContext() { SparkConf conf = new SparkConf(); conf.setMaster(mOptions.getSparkMaster()); conf.setAppName("spark pipeline job"); - conf.set("sun.io.serialization.extendeddebuginfo", "true"); + conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); return new JavaSparkContext(conf); } From a3259c1dce607f66a5077d0c3f1ca48540e9919d Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 22 May 2015 08:38:09 +0100 Subject: [PATCH 067/137] Add a comment explaining use of Coders for serialization. --- .../java/com/cloudera/dataflow/spark/EvaluationContext.java | 2 ++ .../com/cloudera/dataflow/spark/TransformTranslator.java | 6 ++++++ 2 files changed, 8 insertions(+) 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 index 1e71f5d980f6..cb074a1f971a 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -138,6 +138,8 @@ public T getAggregatorValue(String named, Class resultType) { public Iterable get(PCollection pcollection) { @SuppressWarnings("unchecked") JavaRDDLike rdd = (JavaRDDLike) getRDD(pcollection); + // Use a coder to convert the objects in the PCollection to byte arrays, so they + // can be transferred over the network. final Coder coder = pcollection.getCoder(); JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); List clientBytes = bytesRDD.collect(); 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 index dbaf4d945803..340f27f24d95 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -124,6 +124,8 @@ public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContex final Coder keyCoder = coder.getKeyCoder(); final Coder 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>, ?> outRDD = fromPair(toPair(inRDD) .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder)) .groupByKey() @@ -183,6 +185,8 @@ public Tuple2> call(KV kv) { } }); + // Use coders to convert objects in the PCollection to byte arrays, so they + // can be transferred over the network for the shuffle. JavaPairRDD inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair .mapToPair(CoderHelpers.toByteFunction(keyCoder, kviCoder)); @@ -430,6 +434,8 @@ private static TransformEvaluator> create() { @Override public void evaluate(Create transform, EvaluationContext context) { Iterable 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 coder = context.getOutput(transform).getCoder(); JavaRDD rdd = context.getSparkContext().parallelize( CoderHelpers.toByteArrays(elems, coder)); From 7d8f8182bbbce9c2c186df1b60b0c3de3d63aaad Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 28 May 2015 16:42:01 +0100 Subject: [PATCH 068/137] Add HadoopIO for reading from Hadoop InputFormat classes. --- .../cloudera/dataflow/hadoop/HadoopIO.java | 121 ++++++++++++++++++ .../dataflow/spark/TransformTranslator.java | 25 ++++ .../spark/HadoopFileFormatPipelineTest.java | 106 +++++++++++++++ 3 files changed, 252 insertions(+) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.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 new file mode 100644 index 000000000000..c58727c9f860 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -0,0 +1,121 @@ +/* + * 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.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 org.apache.hadoop.mapreduce.lib.input.FileInputFormat; + +public class HadoopIO { + + public static class Read { + public static Bound from(String filepattern) { + return new Bound().from(filepattern); + } + + public static Bound withFormatClass(Class> format) { + return new Bound().withFormatClass(format); + } + + public static Bound withKeyClass(Class key) { + return new Bound().withKeyClass(key); + } + + public static Bound withValueClass(Class value) { + return new Bound().withValueClass(value); + } + + + public static class Bound extends PTransform>> { + + private final String filepattern; + private final Class> formatClass; + private final Class keyClass; + private final Class valueClass; + + public Bound() { + this(null, null, null, null); + } + + public Bound(String filepattern, Class> format, Class key, + Class value) { + this.filepattern = filepattern; + this.formatClass = format; + this.keyClass = key; + this.valueClass = value; + } + + public Bound from(String file) { + return new Bound<>(file, formatClass, keyClass, valueClass); + } + + public Bound withFormatClass(Class> format) { + return new Bound<>(filepattern, format, keyClass, valueClass); + } + + public Bound withKeyClass(Class key) { + return new Bound<>(filepattern, formatClass, key, valueClass); + } + + public Bound withValueClass(Class value) { + return new Bound<>(filepattern, formatClass, keyClass, value); + } + + public String getFilepattern() { + return filepattern; + } + + public Class> getFormatClass() { + return formatClass; + } + + public Class getValueClass() { + return valueClass; + } + + public Class getKeyClass() { + return keyClass; + } + + @Override + public PCollection> apply(PInput input) { + if (filepattern == null) { + throw new IllegalStateException( + "need to set the filepattern of an HadoopIO.Read transform"); + } + if (formatClass == null) { + throw new IllegalStateException( + "need to set the format class of an HadoopIO.Read transform"); + } + if (keyClass == null) { + throw new IllegalStateException( + "need to set the key class of an HadoopIO.Read transform"); + } + if (valueClass == null) { + throw new IllegalStateException( + "need to set the value class of an HadoopIO.Read transform"); + } + + return PCollection.createPrimitiveOutputInternal(WindowingStrategy + .globalDefault()); + } + + } + + } +} 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 index 340f27f24d95..4603ce4c6b4d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -66,6 +66,8 @@ 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. */ @@ -416,6 +418,28 @@ public Tuple2, NullWritable> call(T t) throws Exception { }; } + private static TransformEvaluator> readHadoop() { + return new TransformEvaluator>() { + @Override + public void evaluate(HadoopIO.Read.Bound transform, EvaluationContext context) { + String pattern = transform.getFilepattern(); + JavaSparkContext jsc = context.getSparkContext(); + @SuppressWarnings ("unchecked") + JavaPairRDD file = jsc.newAPIHadoopFile(pattern, + transform.getFormatClass(), + transform.getKeyClass(), transform.getValueClass(), + new Configuration()); + JavaRDD> rdd = file.map(new Function, KV>() { + @Override + public KV call(Tuple2 t2) throws Exception { + return KV.of(t2._1(), t2._2()); + } + }); + context.setOutputRDD(transform, rdd); + } + }; + } + private static TransformEvaluator> window() { return new TransformEvaluator>() { @Override @@ -529,6 +553,7 @@ private static Map, BroadcastHelper> getSideInputs( 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(ParDo.Bound.class, parDo()); EVALUATORS.put(ParDo.BoundMulti.class, multiDo()); EVALUATORS.put(GroupByKey.GroupByKeyOnly.class, gbk()); 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 new file mode 100644 index 000000000000..89b3995b1eed --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -0,0 +1,106 @@ +/* + * 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.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.SequenceFile.Writer; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; +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 testGeneric() throws Exception { + populateFile(); + + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + PCollection> input = (PCollection>) + p.apply(HadoopIO.Read.from(inputFile.getAbsolutePath()) + .withFormatClass(SequenceFileInputFormat.class) + .withKeyClass(IntWritable.class) + .withValueClass(Text.class)); + input.apply(ParDo.of(new TabSeparatedString())) + .apply(TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding()); + SparkPipelineRunner.create().run(p); + + List records = Files.readLines(outputFile, Charsets.UTF_8); + for (int i = 0; i < 5; i++) { + assertEquals(i + "\tvalue-" + i, records.get(i)); + } + } + + private void populateFile() throws IOException { + IntWritable key = new IntWritable(); + Text value = new Text(); + Writer writer = null; + try { + 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); + } + } finally { + IOUtils.closeStream(writer); + } + } + + static class TabSeparatedString extends DoFn, String> { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element().getKey().toString() + "\t" + c.element().getValue().toString()); + } + } + +} From 6d37b90ec2772f171c585e732db738a97e2fce61 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 28 May 2015 22:35:45 +0100 Subject: [PATCH 069/137] Address review feedback. --- .../cloudera/dataflow/hadoop/HadoopIO.java | 46 +++++++++---------- 1 file changed, 22 insertions(+), 24 deletions(-) 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 index c58727c9f860..98d65eee10ee 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -19,11 +19,19 @@ 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 org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -public class HadoopIO { +public final class HadoopIO { + + private HadoopIO() { + } + + public static final class Read { + + private Read() { + } - public static class Read { public static Bound from(String filepattern) { return new Bound().from(filepattern); } @@ -40,7 +48,6 @@ public static Bound withValueClass(Class value) { return new Bound().withValueClass(value); } - public static class Bound extends PTransform>> { private final String filepattern; @@ -48,11 +55,11 @@ public static class Bound extends PTransform> private final Class keyClass; private final Class valueClass; - public Bound() { + Bound() { this(null, null, null, null); } - public Bound(String filepattern, Class> format, Class key, + Bound(String filepattern, Class> format, Class key, Class value) { this.filepattern = filepattern; this.formatClass = format; @@ -94,25 +101,16 @@ public Class getKeyClass() { @Override public PCollection> apply(PInput input) { - if (filepattern == null) { - throw new IllegalStateException( - "need to set the filepattern of an HadoopIO.Read transform"); - } - if (formatClass == null) { - throw new IllegalStateException( - "need to set the format class of an HadoopIO.Read transform"); - } - if (keyClass == null) { - throw new IllegalStateException( - "need to set the key class of an HadoopIO.Read transform"); - } - if (valueClass == null) { - throw new IllegalStateException( - "need to set the value class of an HadoopIO.Read transform"); - } - - return PCollection.createPrimitiveOutputInternal(WindowingStrategy - .globalDefault()); + Preconditions.checkNotNull(filepattern, + "need to set the filepattern of an HadoopIO.Read transform"); + Preconditions.checkNotNull(formatClass, + "need to set the format class of an HadoopIO.Read transform"); + Preconditions.checkNotNull(keyClass, + "need to set the key class of an HadoopIO.Read transform"); + Preconditions.checkNotNull(valueClass, + "need to set the value class of an HadoopIO.Read transform"); + + return PCollection.createPrimitiveOutputInternal(WindowingStrategy.globalDefault()); } } From 70b41e302f61ec9dd1219a3af646a74b68f49f27 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 5 Jun 2015 09:25:38 +0200 Subject: [PATCH 070/137] Remove user-dependent configuration from maven-gpg-plugin. (This should go in a user's settings.xml.) --- runners/spark/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 11421de0c215..6fb33b29ce4b 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -154,10 +154,6 @@ License. - - Sean Owen (Cloudera) <sowen@cloudera.com> - gpg2 - org.apache.rat From c04a5a216279055c94de114ca6faac0af988acf0 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 5 Jun 2015 09:32:04 +0200 Subject: [PATCH 071/137] [maven-release-plugin] prepare release spark-dataflow-0.1.0 --- runners/spark/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 6fb33b29ce4b..38ce3fb8f586 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.0.2-SNAPSHOT + 0.1.0 jar @@ -241,7 +241,7 @@ License. true spark-app - + @@ -378,7 +378,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.1.0 From 0a599c47ed34b6444eaf7fcc3eba1f471799c8ec Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 5 Jun 2015 09:32:10 +0200 Subject: [PATCH 072/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 38ce3fb8f586..af28a2b0015e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.1.0 + 0.1.1-SNAPSHOT jar @@ -378,7 +378,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.1.0 + HEAD From 5f8fc59f28a95e94ad08a08c47f43180196aa50b Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 5 Jun 2015 09:39:27 +0200 Subject: [PATCH 073/137] Update README with latest release. Add brief release instructions. --- runners/spark/README.md | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 6ca5b3da01ed..9bd66ca6cfa3 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.0.1 + 0.1.0 and are hosted in Cloudera's repository at: @@ -100,5 +100,14 @@ Check the output by running: hadoop fs -tail out-00000-of-00002 +## How to Release + +Committers can release the project using the standard [Maven Release Plugin](http://maven.apache.org/maven-release/maven-release-plugin/) commands: + + mvn release:prepare + mvn release:perform -Darguments="-Dgpg.passphrase=XXX" + +Note that you will need a [public GPG key](http://www.apache.org/dev/openpgp.html). + [![Build Status](https://travis-ci.org/cloudera/spark-dataflow.png?branch=master)](https://travis-ci.org/cloudera/spark-dataflow) -[![codecov.io](https://codecov.io/github/cloudera/spark-dataflow/coverage.svg?branch=master)](https://codecov.io/github/cloudera/spark-dataflow?branch=master) \ No newline at end of file +[![codecov.io](https://codecov.io/github/cloudera/spark-dataflow/coverage.svg?branch=master)](https://codecov.io/github/cloudera/spark-dataflow?branch=master) From a35ea6a02172fc4975878c3e5127a6cd882f3f3b Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 17 Jun 2015 10:36:36 +0100 Subject: [PATCH 074/137] Compute PCollections that are created only for the side effects in their computation. This is done by DataflowAssert, for example. --- runners/spark/pom.xml | 6 ++ .../dataflow/spark/EvaluationContext.java | 29 +++++++--- .../dataflow/spark/SparkPipelineRunner.java | 6 ++ .../dataflow/spark/SerializationTest.java | 16 ++++++ .../dataflow/spark/SideEffectsTest.java | 55 +++++++++++++++++++ 5 files changed, 105 insertions(+), 7 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index af28a2b0015e..d002426b49e5 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -330,6 +330,12 @@ License. 4.12 test + + org.hamcrest + hamcrest-all + 1.3 + test + 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 index cb074a1f971a..ff229486f668 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -15,8 +15,8 @@ package com.cloudera.dataflow.spark; -import java.util.HashMap; -import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -44,10 +44,11 @@ public class EvaluationContext implements EvaluationResult { private final Pipeline pipeline; private final SparkRuntimeContext runtime; private final CoderRegistry registry; - private final Map> rdds = new HashMap<>(); - private final Set multireads = new HashSet<>(); - private final Map pobjects = new HashMap<>(); - private final Map>> pview = new HashMap<>(); + private final Map> rdds = new LinkedHashMap<>(); + private final Set> leafRdds = new LinkedHashSet<>(); + private final Set multireads = new LinkedHashSet<>(); + private final Map pobjects = new LinkedHashMap<>(); + private final Map>> pview = new LinkedHashMap<>(); public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; @@ -81,7 +82,7 @@ O getOutput(PTransform transform) { } void setOutputRDD(PTransform transform, JavaRDDLike rdd) { - rdds.put((PValue) getOutput(transform), rdd); + setRDD((PValue) getOutput(transform), rdd); } void setPView(PValue view, Iterable> value) { @@ -90,6 +91,7 @@ void setPView(PValue view, Iterable> value) { JavaRDDLike getRDD(PValue pvalue) { JavaRDDLike rdd = rdds.get(pvalue); + leafRdds.remove(rdd); if (multireads.contains(pvalue)) { // Ensure the RDD is marked as cached rdd.rdd().cache(); @@ -101,6 +103,7 @@ void setPView(PValue view, Iterable> value) { void setRDD(PValue pvalue, JavaRDDLike rdd) { rdds.put(pvalue, rdd); + leafRdds.add(rdd); } JavaRDDLike getInputRDD(PTransform transform) { @@ -112,6 +115,18 @@ Iterable> getPCollectionView(PCollectionView 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). + */ + void computeOutputs() { + for (JavaRDDLike rdd : leafRdds) { + rdd.rdd().cache(); // cache so that any subsequent get() is cheap + rdd.count(); // force the RDD to be computed + } + } + @Override public T get(PValue value) { if (pobjects.containsKey(value)) { 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 index daa30f55bf58..1d7f012ad6ff 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -101,9 +101,15 @@ private SparkPipelineRunner(SparkPipelineOptions options) { @Override public EvaluationResult run(Pipeline pipeline) { + LOG.info("Executing pipeline using the SparkPipelineRunner."); + JavaSparkContext jsc = getContext(); EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); + ctxt.computeOutputs(); + + LOG.info("Pipeline execution complete."); + return ctxt; } 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 index dff4bbb4da8a..fc8a2c01846a 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -44,9 +44,25 @@ public class SerializationTest { public static class StringHolder { // not serializable private 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; 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 new file mode 100644 index 000000000000..7292bf083994 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java @@ -0,0 +1,55 @@ +/* + * 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 com.google.cloud.dataflow.sdk.values.PCollection; +import java.io.Serializable; +import java.net.URI; +import org.junit.Test; + +import static org.junit.Assert.fail; + +public class SideEffectsTest implements Serializable { + @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)); + + PCollection strings = pipeline.apply(Create.of("a")); + PCollection output = strings.apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + throw new IllegalStateException("Side effect"); + } + })); + + try { + pipeline.run(); + fail("Run should thrown an exception"); + } catch (Exception e) { + // expected + e.printStackTrace(); + } + } +} From 2bb6c11652d039a7d7e126ae80a6dd2345d8d51a Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 17 Jun 2015 17:34:48 +0100 Subject: [PATCH 075/137] Wrap failures in pipeline.run() in a RuntimeException. Ideally the RuntimeException's cause exception would be the one thrown in the task (from the user's code), but Spark doesn't send the exceptions back to the driver, so this isn't currently possible. --- .../dataflow/spark/SparkPipelineRunner.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) 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 index 1d7f012ad6ff..04bc74085c49 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -101,16 +101,20 @@ private SparkPipelineRunner(SparkPipelineOptions options) { @Override public EvaluationResult run(Pipeline pipeline) { - LOG.info("Executing pipeline using the SparkPipelineRunner."); + try { + LOG.info("Executing pipeline using the SparkPipelineRunner."); - JavaSparkContext jsc = getContext(); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); - pipeline.traverseTopologically(new Evaluator(ctxt)); - ctxt.computeOutputs(); + JavaSparkContext jsc = getContext(); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); + pipeline.traverseTopologically(new Evaluator(ctxt)); + ctxt.computeOutputs(); - LOG.info("Pipeline execution complete."); + LOG.info("Pipeline execution complete."); - return ctxt; + return ctxt; + } catch (Exception e) { + throw new RuntimeException(e); // wrap a SparkException in a RuntimeException + } } private JavaSparkContext getContext() { From 45d3e61c5d94000ae6d33085dfc8c53fd279a94b Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 17 Jun 2015 18:43:44 +0100 Subject: [PATCH 076/137] Implement DoFn.Context.getPipelineOptions(). --- .../dataflow/spark/SparkRuntimeContext.java | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) 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 index c7bdc560b958..ca9fb008cece 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -15,10 +15,13 @@ package com.cloudera.dataflow.spark; +import java.io.IOException; import java.io.Serializable; 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.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; @@ -45,6 +48,9 @@ public class SparkRuntimeContext implements Serializable { * An accumulator that is a map from names to aggregators. */ private final Accumulator accum; + + private final String serializedPipelineOptions; + /** * Map fo names to dataflow aggregators. */ @@ -53,6 +59,23 @@ public class SparkRuntimeContext implements Serializable { 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); + } } /** @@ -68,8 +91,7 @@ public T getAggregatorValue(String aggregatorName, Class typeClass) { } public synchronized PipelineOptions getPipelineOptions() { - //TODO: Support this. - throw new UnsupportedOperationException("getPipelineOptions is not yet supported."); + return deserializePipelineOptions(serializedPipelineOptions); } /** From 48347caf18be1e6026b6863c7bbf4c557ed10273 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 18 Jun 2015 15:55:43 +0100 Subject: [PATCH 077/137] [maven-release-plugin] prepare release spark-dataflow-0.1.1 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index d002426b49e5..ebbe25a65a3d 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.1.1-SNAPSHOT + 0.1.1 jar @@ -384,7 +384,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.1.1 From ced9b30a3d80c22c3c1e9a8c0f5862f3d97cf287 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 18 Jun 2015 15:55:49 +0100 Subject: [PATCH 078/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index ebbe25a65a3d..a2de2f5386ee 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.1.1 + 0.1.2-SNAPSHOT jar @@ -384,7 +384,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.1.1 + HEAD From f85beb3d1f3cec44d97fb03bc7b8c64163d29972 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 18 Jun 2015 16:16:33 +0100 Subject: [PATCH 079/137] Update README with 0.1.1 minor release. --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 9bd66ca6cfa3..d4de19f02c05 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.1.0 + 0.1.1 and are hosted in Cloudera's repository at: From b0312a1424bcab3b17ae3b89c3abb80705f6ed0c Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 19 Jun 2015 13:48:53 +0100 Subject: [PATCH 080/137] Allow tests to share a single Spark context. It's not possible to have multiple Spark contexts per JVM (even if not running concurrently), and while one solution is to fork a new JVM per test class, it's not always possible or desirable to do this - if for example the test suite has a dataflow test per test method, or the test suite has many very small tests. This JIRA has more background and discussion about the JVM limitation: https://issues.apache.org/jira/browse/SPARK-2243 This change makes it possible to share a SparkContext when running tests that use the SparkPipelineRunner by setting the system property 'dataflow.spark.test.reuseSparkContext' to 'true'. --- .../dataflow/spark/EvaluationContext.java | 2 +- .../dataflow/spark/SparkContextFactory.java | 66 +++++++++++++++++++ .../dataflow/spark/SparkPipelineRunner.java | 12 +--- .../dataflow/spark/AvroPipelineTest.java | 3 +- .../cloudera/dataflow/spark/DeDupTest.java | 3 +- .../dataflow/spark/EmptyInputTest.java | 5 +- .../spark/HadoopFileFormatPipelineTest.java | 3 +- .../dataflow/spark/NumShardsTest.java | 3 +- .../dataflow/spark/SerializationTest.java | 4 +- .../dataflow/spark/SimpleWordCountTest.java | 4 +- .../cloudera/dataflow/spark/TfIdfTest.java | 3 +- 11 files changed, 85 insertions(+), 23 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.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 index ff229486f668..6f1e8693ff3a 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -168,7 +168,7 @@ public T apply(byte[] bytes) { @Override public void close() { - jsc.stop(); + SparkContextFactory.stopSparkContext(jsc); } @Override 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 new file mode 100644 index 000000000000..483899a156f3 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 dataflow.spark.test.reuseSparkContext is set to + * true then the Spark context will be reused for dataflow pipelines. + * This property should only be enabled for tests. + */ + public 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) { + if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) { + if (sparkContext == null) { + sparkContext = createSparkContext(master); + 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); + } + } + + public static synchronized void stopSparkContext(JavaSparkContext context) { + if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) { + context.stop(); + } + } + + private static JavaSparkContext createSparkContext(String master) { + SparkConf conf = new SparkConf(); + conf.setMaster(master); + conf.setAppName("spark pipeline job"); + conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); + return new JavaSparkContext(conf); + } +} 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 index 04bc74085c49..6d6e69fcaea6 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -22,9 +22,7 @@ import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.values.PValue; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.serializer.KryoSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,7 +102,7 @@ public EvaluationResult run(Pipeline pipeline) { try { LOG.info("Executing pipeline using the SparkPipelineRunner."); - JavaSparkContext jsc = getContext(); + JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster()); EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); pipeline.traverseTopologically(new Evaluator(ctxt)); ctxt.computeOutputs(); @@ -117,14 +115,6 @@ public EvaluationResult run(Pipeline pipeline) { } } - private JavaSparkContext getContext() { - SparkConf conf = new SparkConf(); - conf.setMaster(mOptions.getSparkMaster()); - conf.setAppName("spark pipeline job"); - conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); - return new JavaSparkContext(conf); - } - private static final class Evaluator implements Pipeline.PipelineVisitor { private final EvaluationContext ctxt; 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 index 673244e4a10c..95f100c1c390 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java @@ -67,7 +67,8 @@ public void testGeneric() throws Exception { PCollection input = p.apply( AvroIO.Read.from(inputFile.getAbsolutePath()).withSchema(schema)); input.apply(AvroIO.Write.to(outputDir.getAbsolutePath()).withSchema(schema)); - SparkPipelineRunner.create().run(p); + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); List records = readGenericFile(); assertEquals(Lists.newArrayList(savedRecord), records); 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 index 7179c166aad2..7495aeb3e2d3 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DeDupTest.java @@ -49,6 +49,7 @@ public void testRun() throws Exception { DataflowAssert.that(output).containsInAnyOrder(EXPECTED_SET); - p.run(); + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); } } 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 index 066af84e5289..6c89ca105c74 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/EmptyInputTest.java @@ -38,8 +38,9 @@ public void test() throws Exception { PCollection inputWords = p.apply(Create.of(empty)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(Combine.globally(new ConcatWords())); - EvaluationResult run = SparkPipelineRunner.create(options).run(p); - assertEquals("", Iterables.getOnlyElement(run.get(output))); + EvaluationResult res = SparkPipelineRunner.create().run(p); + assertEquals("", Iterables.getOnlyElement(res.get(output))); + res.close(); } public static class ConcatWords implements SerializableFunction, String> { 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 index 89b3995b1eed..9aa634e444ca 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -70,7 +70,8 @@ public void testGeneric() throws Exception { .withValueClass(Text.class)); input.apply(ParDo.of(new TabSeparatedString())) .apply(TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding()); - SparkPipelineRunner.create().run(p); + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); List records = Files.readLines(outputFile, Charsets.UTF_8); for (int i = 0; i < 5; i++) { 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 index ea6fcfc33a93..8985e663f2a4 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java @@ -64,7 +64,8 @@ public void testText() throws Exception { PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(new WordCount.CountWords()); output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt")); - p.run(); + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); int count = 0; Set expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2"); 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 index fc8a2c01846a..bf63bfaacb16 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -123,8 +123,8 @@ public void testRun() throws Exception { DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); - p.run(); - + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); } /** 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 index df403eb09680..438231f5caa9 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -52,8 +52,8 @@ public void testRun() throws Exception { DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); - p.run(); - + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); } /** 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 index 48620607fc73..7e590ab96328 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -98,7 +98,8 @@ public void testRun() throws Exception { .apply(new ComputeTfIdf()) .apply(new WriteTfIdf(outputDir.toURI().toString())); - SparkPipelineRunner.create().run(pipeline); + EvaluationResult res = SparkPipelineRunner.create().run(pipeline); + res.close(); for (File f : tmpDir.getRoot().listFiles(new FileFilter() { @Override From 148979ff9f60896d3373e598ceaee0570f1e74a2 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 11 Jun 2015 11:26:07 +0100 Subject: [PATCH 081/137] Update to dataflow 0.4.150602. --- runners/spark/pom.xml | 5 +- .../cloudera/dataflow/hadoop/HadoopIO.java | 3 +- .../cloudera/dataflow/spark/DoFnFunction.java | 101 ++++- .../dataflow/spark/EvaluationContext.java | 29 +- .../dataflow/spark/MultiDoFnFunction.java | 22 +- .../dataflow/spark/SparkPipelineRunner.java | 15 +- .../dataflow/spark/SparkRuntimeContext.java | 80 ++-- .../dataflow/spark/TransformTranslator.java | 12 +- .../spark/aggregators/NamedAggregators.java | 17 +- .../spark/MultiOutputWordCountTest.java | 14 +- .../dataflow/spark/SerializationTest.java | 13 +- .../dataflow/spark/SimpleWordCountTest.java | 8 +- .../cloudera/dataflow/spark/TfIdfTest.java | 374 +----------------- 13 files changed, 230 insertions(+), 463 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index a2de2f5386ee..fb70e910b775 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,6 +23,7 @@ License. UTF-8 1.7 1.3.1 + 0.4.150602 @@ -288,7 +289,7 @@ License. com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - 0.4.150414 + ${google-cloud-dataflow-version} @@ -300,7 +301,7 @@ License. com.google.cloud.dataflow google-cloud-dataflow-java-examples-all - 0.4.150414 + ${google-cloud-dataflow-version} 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 index 98d65eee10ee..434a1326b663 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -110,7 +110,8 @@ public PCollection> apply(PInput input) { Preconditions.checkNotNull(valueClass, "need to set the value class of an HadoopIO.Read transform"); - return PCollection.createPrimitiveOutputInternal(WindowingStrategy.globalDefault()); + return PCollection.createPrimitiveOutputInternal(input.getPipeline(), + WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); } } 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 index 875297c81c3d..7ae52b033f7c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -15,19 +15,25 @@ package com.cloudera.dataflow.spark; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.LinkedList; import java.util.List; 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.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.TimerManager; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.spark.api.java.function.FlatMapFunction; @@ -44,6 +50,9 @@ class DoFnFunction implements FlatMapFunction, O> { private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); + private static final Collection GLOBAL_WINDOWS = + Collections.singletonList(GlobalWindow.INSTANCE); + private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -67,6 +76,7 @@ public Iterable call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction); //setup mFunction.startBundle(ctxt); + ctxt.setup(); //operation while (iter.hasNext()) { ctxt.element = iter.next(); @@ -86,6 +96,10 @@ private class ProcCtxt extends DoFn.ProcessContext { fn.super(); } + void setup() { + super.setupDelegateAggregators(); + } + @Override public PipelineOptions getPipelineOptions() { return mRuntimeContext.getPipelineOptions(); @@ -123,29 +137,17 @@ public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant insta } @Override - public Aggregator createAggregator( + public Aggregator createAggregatorInternal( String named, - Combine.CombineFn combineFn) { + Combine.CombineFn combineFn) { return mRuntimeContext.createAggregator(named, combineFn); } - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return mRuntimeContext.createAggregator(named, sfunc); - } - @Override public I element() { return element; } - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - @Override public void outputWithTimestamp(O output, Instant timestamp) { output(output); @@ -158,12 +160,77 @@ public Instant timestamp() { @Override public BoundedWindow window() { - return null; + return GlobalWindow.INSTANCE; } @Override public WindowingInternals windowingInternals() { - return null; + return new WindowingInternals() { + + @Override + public Collection windows() { + return GLOBAL_WINDOWS; + } + + @Override + public void outputWindowedValue(O output, Instant timestamp, Collection windows) { + output(output); + } + + @Override + public KeyedState keyedState() { + throw new UnsupportedOperationException( + "WindowingInternals#keyedState() is not yet supported."); + + } + + @Override + public void store(CodedTupleTag tag, T value, Instant timestamp) + throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#store() is not yet supported."); + } + + @Override + public void writeToTagList(CodedTupleTag tag, T value) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#writeToTagList() is not yet supported."); + } + + @Override + public void deleteTagList(CodedTupleTag tag) { + throw new UnsupportedOperationException( + "WindowingInternals#deleteTagList() is not yet supported."); + } + + @Override + public Iterable readTagList(CodedTupleTag tag) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#readTagList() is not yet supported."); + } + + @Override + public Map, Iterable> readTagList(List> tags) + throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#readTagList() is not yet supported."); + } + + @Override + public TimerManager getTimerManager() { + throw new UnsupportedOperationException( + "WindowingInternals#getTimerManager() is not yet supported."); + } + + @Override + public void writePCollectionViewData(TupleTag tag, + Iterable> data, Coder elemCoder) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#writePCollectionViewData() is not yet supported."); + } + }; } + } } 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 index 6f1e8693ff3a..c7aa7c6366fb 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -15,6 +15,8 @@ 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; @@ -24,6 +26,10 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +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; @@ -36,6 +42,7 @@ import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; + /** * Evaluation context allows us to define how pipeline instructions. */ @@ -49,6 +56,7 @@ public class EvaluationContext implements EvaluationResult { private final Set multireads = new LinkedHashSet<>(); private final Map pobjects = new LinkedHashMap<>(); private final Map>> pview = new LinkedHashMap<>(); + private AppliedPTransform currentTransform; public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; @@ -69,15 +77,23 @@ SparkRuntimeContext getRuntimeContext() { return runtime; } + void setCurrentTransform(AppliedPTransform transform) { + this.currentTransform = transform; + } + I getInput(PTransform transform) { + checkArgument(currentTransform != null && currentTransform.getTransform() == transform, + "can only be called with current transform"); @SuppressWarnings("unchecked") - I input = (I) pipeline.getInput(transform); + I input = (I) currentTransform.getInput(); return input; } O getOutput(PTransform transform) { + checkArgument(currentTransform != null && currentTransform.getTransform() == transform, + "can only be called with current transform"); @SuppressWarnings("unchecked") - O output = (O) pipeline.getOutput(transform); + O output = (O) currentTransform.getOutput(); return output; } @@ -107,7 +123,7 @@ void setRDD(PValue pvalue, JavaRDDLike rdd) { } JavaRDDLike getInputRDD(PTransform transform) { - return getRDD((PValue) pipeline.getInput(transform)); + return getRDD((PValue) getInput(transform)); } @@ -149,6 +165,13 @@ public T getAggregatorValue(String named, Class resultType) { return runtime.getAggregatorValue(named, resultType); } + @Override + public AggregatorValues getAggregatorValues(Aggregator aggregator) + throws AggregatorRetrievalException { + //TODO: Support this. + throw new UnsupportedOperationException("getAggregatorValues is not yet supported."); + } + @Override public Iterable get(PCollection pcollection) { @SuppressWarnings("unchecked") 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 index 38bccc428964..243fa67a6b68 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -22,7 +22,6 @@ 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.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; @@ -68,6 +67,7 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT public Iterable, Object>> call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction); mFunction.startBundle(ctxt); + ctxt.setup(); while (iter.hasNext()) { ctxt.element = iter.next(); mFunction.processElement(ctxt); @@ -91,6 +91,10 @@ private class ProcCtxt extends DoFn.ProcessContext { fn.super(); } + void setup() { + super.setupDelegateAggregators(); + } + @Override public PipelineOptions getPipelineOptions() { return mRuntimeContext.getPipelineOptions(); @@ -121,29 +125,17 @@ public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant insta } @Override - public Aggregator createAggregator( + public Aggregator createAggregatorInternal( String named, - Combine.CombineFn combineFn) { + Combine.CombineFn combineFn) { return mRuntimeContext.createAggregator(named, combineFn); } - @Override - public Aggregator createAggregator( - String named, - SerializableFunction, AO> sfunc) { - return mRuntimeContext.createAggregator(named, sfunc); - } - @Override public I element() { return element; } - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException(); - } - @Override public void outputWithTimestamp(O output, Instant timestamp) { output(output); 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 index 6d6e69fcaea6..287be1d0424c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -20,7 +20,10 @@ 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.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 com.google.cloud.dataflow.sdk.values.PValue; import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; @@ -159,7 +162,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { if (inTranslatedCompositeNode() && node.equals(currentTranslatedCompositeNode)) { LOG.info("Post-visiting directly-translatable composite transform: '{}'", node.getFullName()); - doVisitTransform(node.getTransform()); + doVisitTransform(node); currentTranslatedCompositeNode = null; } } @@ -170,15 +173,21 @@ public void visitTransform(TransformTreeNode node) { LOG.info("Skipping '{}'; already in composite transform.", node.getFullName()); return; } - doVisitTransform(node.getTransform()); + doVisitTransform(node); } - private void doVisitTransform(PT transform) { + private void doVisitTransform(TransformTreeNode node) { + PT transform = (PT) node.getTransform(); @SuppressWarnings("unchecked") TransformEvaluator evaluator = (TransformEvaluator) TransformTranslator.getTransformEvaluator(transform.getClass()); LOG.info("Evaluating {}", transform); + AppliedPTransform appliedTransform = + AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), + (PTransform) transform); + ctxt.setCurrentTransform(appliedTransform); evaluator.evaluate(transform, ctxt); + ctxt.setCurrentTransform(null); } @Override 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 index ca9fb008cece..35e3937d2278 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -23,6 +23,7 @@ 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; @@ -32,7 +33,7 @@ import com.google.cloud.dataflow.sdk.transforms.Min; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Sum; -import com.google.common.reflect.TypeToken; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; @@ -54,7 +55,7 @@ public class SparkRuntimeContext implements Serializable { /** * Map fo names to dataflow aggregators. */ - private final Map> aggregators = new HashMap<>(); + private final Map> aggregators = new HashMap<>(); private transient CoderRegistry coderRegistry; SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { @@ -103,16 +104,16 @@ public synchronized PipelineOptions getPipelineOptions() { * @param Type of aggregator outputs. * @return Specified aggregator */ - public synchronized Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, SerializableFunction, OUT> sfunc) { @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { NamedAggregators.SerFunctionState state = new NamedAggregators .SerFunctionState<>(sfunc); accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator<>(state); + aggregator = new SparkAggregator<>(named, state); aggregators.put(named, aggregator); } return aggregator; @@ -128,11 +129,11 @@ public synchronized Aggregator createAggregator( * @param Type of aggregator outputs. * @return Specified aggregator */ - public synchronized Aggregator createAggregator( + public synchronized Aggregator createAggregator( String named, Combine.CombineFn combineFn) { @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); + Aggregator aggregator = (Aggregator) aggregators.get(named); if (aggregator == null) { @SuppressWarnings("unchecked") NamedAggregators.CombineFunctionState state = @@ -141,7 +142,7 @@ public synchronized Aggregator createAggregator( (Coder) getCoder(combineFn), this); accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator<>(state); + aggregator = new SparkAggregator<>(named, state); aggregators.put(named, aggregator); } return aggregator; @@ -156,27 +157,31 @@ public CoderRegistry getCoderRegistry() { } private Coder getCoder(Combine.CombineFn combiner) { - if (combiner.getClass() == Sum.SumIntegerFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); - } else if (combiner.getClass() == Sum.SumLongFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); - } else if (combiner.getClass() == Sum.SumDoubleFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); - } else if (combiner.getClass() == Min.MinIntegerFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); - } else if (combiner.getClass() == Min.MinLongFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); - } else if (combiner.getClass() == Min.MinDoubleFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); - } else if (combiner.getClass() == Max.MaxIntegerFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Integer.class)); - } else if (combiner.getClass() == Max.MaxLongFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Long.class)); - } else if (combiner.getClass() == Max.MaxDoubleFn.class) { - return getCoderRegistry().getDefaultCoder(TypeToken.of(Double.class)); - } else { - throw new IllegalArgumentException("unsupported combiner in Aggregator: " - + combiner.getClass().getName()); + 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); } } @@ -185,16 +190,29 @@ private Coder getCoder(Combine.CombineFn combiner) { * * @param Type of element fed in to aggregator. */ - private static class SparkAggregator implements Aggregator, Serializable { + private static class SparkAggregator implements Aggregator, Serializable { + private final String name; private final NamedAggregators.State state; - SparkAggregator(NamedAggregators.State state) { + SparkAggregator(String name, NamedAggregators.State 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 getCombineFn() { + //TODO: Support this. + throw new UnsupportedOperationException("getCombineFn is not yet supported."); + } } } 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 index 4603ce4c6b4d..ceeed1351018 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -21,6 +21,7 @@ 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; @@ -103,7 +104,7 @@ private static TransformEvaluator> flatten @SuppressWarnings("unchecked") @Override public void evaluate(Flatten.FlattenPCollectionList transform, EvaluationContext context) { - PCollectionList pcs = (PCollectionList) context.getPipeline().getInput(transform); + PCollectionList pcs = context.getInput(transform); JavaRDD[] rdds = new JavaRDD[pcs.size()]; for (int i = 0; i < rdds.length; i++) { rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); @@ -168,8 +169,13 @@ public void evaluate(Combine.PerKey transform, EvaluationContext cont KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); Coder keyCoder = inputCoder.getKeyCoder(); Coder viCoder = inputCoder.getValueCoder(); - Coder vaCoder = keyed.getAccumulatorCoder( - context.getPipeline().getCoderRegistry(), keyCoder, viCoder); + Coder vaCoder = null; + try { + vaCoder = keyed.getAccumulatorCoder( + context.getPipeline().getCoderRegistry(), keyCoder, viCoder); + } catch (CannotProvideCoderException e) { + throw new IllegalStateException("Could not determine coder for accumulator", e); + } final Coder> kviCoder = KvCoder.of(keyCoder, viCoder); final Coder> kvaCoder = KvCoder.of(keyCoder, vaCoder); 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 index 904f71eebb3e..418e684110ba 100644 --- 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 @@ -22,6 +22,7 @@ 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.cloud.dataflow.sdk.transforms.SerializableFunction; @@ -170,8 +171,12 @@ private void writeObject(ObjectOutputStream oos) throws IOException { oos.writeObject(ctxt); oos.writeObject(combineFn); oos.writeObject(inCoder); - combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .encode(state, oos, Coder.Context.NESTED); + 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") @@ -179,8 +184,12 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound ctxt = (SparkRuntimeContext) ois.readObject(); combineFn = (Combine.CombineFn) ois.readObject(); inCoder = (Coder) ois.readObject(); - state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .decode(ois, Coder.Context.NESTED); + 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); + } } } 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 index 67c9b883470b..b16320d11fba 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -78,22 +78,16 @@ public void testRun() throws Exception { */ static class ExtractWordsFn extends DoFn { - private Aggregator totalWords; - private Aggregator maxWordLength; + private Aggregator totalWords = createAggregator("totalWords", + new Sum.SumIntegerFn()); + private Aggregator maxWordLength = createAggregator("maxWordLength", + new Max.MaxIntegerFn()); private final PCollectionView regex; ExtractWordsFn(PCollectionView regex) { this.regex = regex; } - @Override - public void startBundle(Context ctxt) { - this.totalWords = ctxt.createAggregator("totalWords", - new Sum.SumIntegerFn()); - this.maxWordLength = ctxt.createAggregator("maxWordLength", - new Max.MaxIntegerFn()); - } - @Override public void processElement(ProcessContext c) { String[] words = c.element().split(c.sideInput(regex)); 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 index bf63bfaacb16..40591e59aa1a 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -83,11 +83,6 @@ public StringHolder decode(InputStream inStream, Context context) throws IOExcep return new StringHolder(stringUtf8Coder.decode(inStream, context)); } - @Override - public boolean isDeterministic() { - return true; - } - public static Coder of() { return new StringHolderUtf8Coder(); } @@ -131,12 +126,8 @@ public void testRun() throws Exception { * A DoFn that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { - private Aggregator emptyLines; - - @Override - public void startBundle(Context c) { - emptyLines = c.createAggregator("emptyLines", new Sum.SumLongFn()); - } + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); @Override public void processElement(ProcessContext c) { 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 index 438231f5caa9..0f6db1f7ccb4 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -60,12 +60,8 @@ public void testRun() throws Exception { * A DoFn that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { - private Aggregator emptyLines; - - @Override - public void startBundle(Context c) { - emptyLines = c.createAggregator("emptyLines", new Sum.SumLongFn()); - } + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); @Override public void processElement(ProcessContext c) { 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 index 7e590ab96328..680d8b70492a 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -15,386 +15,46 @@ package com.cloudera.dataflow.spark; +import com.google.cloud.dataflow.examples.TfIdf; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; -import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.Count; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.Flatten; +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.PTransform; -import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; -import com.google.cloud.dataflow.sdk.transforms.Values; -import com.google.cloud.dataflow.sdk.transforms.View; -import com.google.cloud.dataflow.sdk.transforms.WithKeys; -import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; -import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; -import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; 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.PCollectionView; -import com.google.cloud.dataflow.sdk.values.PDone; -import com.google.cloud.dataflow.sdk.values.PInput; -import com.google.cloud.dataflow.sdk.values.TupleTag; -import com.google.common.base.Charsets; -import com.google.common.io.Files; -import com.google.common.io.Resources; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; import java.net.URI; -import java.util.HashSet; -import java.util.Locale; -import java.util.Set; -import org.junit.Before; -import org.junit.Rule; +import java.util.Arrays; import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * A test based on {@code TfIdf} from the SDK. */ public class TfIdfTest { - private File inputDir; - private File outputDir; - - @Rule - public final TemporaryFolder tmpDir = new TemporaryFolder(); - - @Before - public void setUp() throws IOException { - inputDir = tmpDir.newFolder("in"); - copy("pg1112.txt", inputDir); - copy("pg2264.txt", inputDir); - outputDir = tmpDir.newFolder("out"); - outputDir.delete(); - } - - private static void copy(String resourceName, File dir) throws IOException { - File dest = new File(dir, resourceName); - Resources.copy(Resources.getResource(resourceName), Files.asByteSink(dest).openStream()); - } - @Test - public void testRun() throws Exception { + public void testTfIdf() throws Exception { Pipeline pipeline = Pipeline.create(PipelineOptionsFactory.create()); pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - pipeline - .apply(new ReadDocuments(listInputDocuments())) - .apply(new ComputeTfIdf()) - .apply(new WriteTfIdf(outputDir.toURI().toString())); - - EvaluationResult res = SparkPipelineRunner.create().run(pipeline); - res.close(); - - for (File f : tmpDir.getRoot().listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - return pathname.getName().startsWith("out-"); - } - })) { - for (String line : Files.readLines(f, Charsets.UTF_8)) { - if (line.startsWith("love,")) { - assertTrue("Incorrect term frequency", line.endsWith("0.003484")); - return; - } - } - - } - fail("Could not find 'love' in output."); - } + PCollection>> 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()); - public Set listInputDocuments() { - Set uris = new HashSet<>(); - File directory = inputDir; - for (String entry : directory.list()) { - File path = new File(directory, entry); - uris.add(path.toURI()); - } - return uris; - } - - /** - * Reads the documents at the provided uris and returns all lines - * from the documents tagged with which document they are from. - */ - public static class ReadDocuments - extends PTransform>> { - - private final Iterable uris; - - public ReadDocuments(Iterable uris) { - this.uris = uris; - } - - @Override - public Coder getDefaultOutputCoder() { - return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); - } - - @Override - public PCollection> apply(PInput input) { - Pipeline pipeline = input.getPipeline(); - - // Create one TextIO.Read transform for each document - // and add its output to a PCollectionList - PCollectionList> urisToLines = - PCollectionList.empty(pipeline); - - // TextIO.Read supports: - // - file: URIs and paths locally - // - gs: URIs on the service - for (URI uri : uris) { - String uriString; - if ("file".equals(uri.getScheme())) { - uriString = new File(uri).getPath(); - } else { - uriString = uri.toString(); - } - - PCollection> oneUriToLines = pipeline - .apply(TextIO.Read.from(uriString) - .named("TextIO.Read(" + uriString + ")")) - .apply(WithKeys.of(uri).withName("WithKeys(" + uriString + ")")); - - urisToLines = urisToLines.and(oneUriToLines); - } - - return urisToLines.apply(Flatten.>create()); - } - } + PCollection words = wordToUriAndTfIdf + .apply(Keys.create()) + .apply(RemoveDuplicates.create()); - /** - * A transform containing a basic TF-IDF pipeline. The input consists of KV objects - * where the key is the document's URI and the value is a piece - * of the document's content. The output is mapping from terms to - * scores for each document URI. - */ - public static class ComputeTfIdf - extends PTransform>, PCollection>>> { + DataflowAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d")); - @Override - public PCollection>> apply( - PCollection> uriToContent) { - - // Compute the total number of documents, and - // prepare this singleton PCollectionView for - // use as a side input. - final PCollectionView totalDocuments = - uriToContent - .apply(Keys.create().withName("GetURIs")) - .apply(RemoveDuplicates.create().withName("RemoveDuplicateDocs")) - .apply(Count.globally()) - .apply(View.asSingleton()); - - // Create a collection of pairs mapping a URI to each - // of the words in the document associated with that that URI. - PCollection> uriToWords = uriToContent - .apply(ParDo.named("SplitWords").of( - new DoFn, KV>() { - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey(); - String line = c.element().getValue(); - for (String word : line.split("\\W+")) { - if (LOG.isDebugEnabled()) { - // Log messages when the word "love" is found. - if ("love".equals(word.toLowerCase(Locale.ENGLISH))) { - LOG.debug("Found {}", word.toLowerCase(Locale.ENGLISH)); - } - } - - if (!word.isEmpty()) { - c.output(KV.of(uri, word.toLowerCase(Locale.ENGLISH))); - } - } - } - })); - - // Compute a mapping from each word to the total - // number of documents in which it appears. - PCollection> wordToDocCount = uriToWords - .apply(RemoveDuplicates.>create().withName("RemoveDuplicateWords")) - .apply(Values.create()) - .apply(Count.perElement().withName("CountDocs")); - - // Compute a mapping from each URI to the total - // number of words in the document associated with that URI. - PCollection> uriToWordTotal = uriToWords - .apply(Keys.create().withName("GetURIs2")) - .apply(Count.perElement().withName("CountWords")); - - // Count, for each (URI, word) pair, the number of - // occurrences of that word in the document associated - // with the URI. - PCollection, Long>> uriAndWordToCount = uriToWords - .apply(Count.>perElement().withName("CountWordDocPairs")); - - // Adjust the above collection to a mapping from - // (URI, word) pairs to counts into an isomorphic mapping - // from URI to (word, count) pairs, to prepare for a join - // by the URI key. - PCollection>> uriToWordAndCount = uriAndWordToCount - .apply(ParDo.named("ShiftKeys").of( - new DoFn, Long>, KV>>() { - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey().getKey(); - String word = c.element().getKey().getValue(); - Long occurrences = c.element().getValue(); - c.output(KV.of(uri, KV.of(word, occurrences))); - } - })); - - // Prepare to join the mapping of URI to (word, count) pairs with - // the mapping of URI to total word counts, by associating - // each of the input PCollection> with - // a tuple tag. Each input must have the same key type, URI - // in this case. The type parameter of the tuple tag matches - // the types of the values for each collection. - final TupleTag wordTotalsTag = new TupleTag<>(); - final TupleTag> wordCountsTag = new TupleTag<>(); - KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple - .of(wordTotalsTag, uriToWordTotal) - .and(wordCountsTag, uriToWordAndCount); - - // Perform a CoGroupByKey (a sort of pre-join) on the prepared - // inputs. This yields a mapping from URI to a CoGbkResult - // (CoGroupByKey Result). The CoGbkResult is a mapping - // from the above tuple tags to the values in each input - // associated with a particular URI. In this case, each - // KV group a URI with the total number of - // words in that document as well as all the (word, count) - // pairs for particular words. - PCollection> uriToWordAndCountAndTotal = coGbkInput - .apply(CoGroupByKey.create().withName("CoGroupByURI")); - - // Compute a mapping from each word to a (URI, term frequency) - // pair for each URI. A word's term frequency for a document - // is simply the number of times that word occurs in the document - // divided by the total number of words in the document. - PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal - .apply(ParDo.named("ComputeTermFrequencies").of( - new DoFn, KV>>() { - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey(); - Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); - - for (KV wordAndCount - : c.element().getValue().getAll(wordCountsTag)) { - String word = wordAndCount.getKey(); - Long wordCount = wordAndCount.getValue(); - Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); - c.output(KV.of(word, KV.of(uri, termFrequency))); - } - } - })); - - // Compute a mapping from each word to its document frequency. - // A word's document frequency in a corpus is the number of - // documents in which the word appears divided by the total - // number of documents in the corpus. Note how the total number of - // documents is passed as a side input; the same value is - // presented to each invocation of the DoFn. - PCollection> wordToDf = wordToDocCount - .apply(ParDo - .named("ComputeDocFrequencies") - .withSideInputs(totalDocuments) - .of(new DoFn, KV>() { - @Override - public void processElement(ProcessContext c) { - String word = c.element().getKey(); - Long documentCount = c.element().getValue(); - Long documentTotal = c.sideInput(totalDocuments); - Double documentFrequency = documentCount.doubleValue() - / documentTotal.doubleValue(); - - c.output(KV.of(word, documentFrequency)); - } - })); - - // Join the term frequency and document frequency - // collections, each keyed on the word. - final TupleTag> tfTag = new TupleTag<>(); - final TupleTag dfTag = new TupleTag<>(); - PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple - .of(tfTag, wordToUriAndTf) - .and(dfTag, wordToDf) - .apply(CoGroupByKey.create()); - - // Compute a mapping from each word to a (URI, TF-IDF) score - // for each URI. There are a variety of definitions of TF-IDF - // ("term frequency - inverse document frequency") score; - // here we use a basic version that is the term frequency - // divided by the log of the document frequency. - - return wordToUriAndTfAndDf - .apply(ParDo.named("ComputeTfIdf").of( - new DoFn, KV>>() { - @Override - public void processElement(ProcessContext c) { - String word = c.element().getKey(); - Double df = c.element().getValue().getOnly(dfTag); - - for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { - URI uri = uriAndTf.getKey(); - Double tf = uriAndTf.getValue(); - Double tfIdf = tf * Math.log(1 / df); - c.output(KV.of(word, KV.of(uri, tfIdf))); - } - } - })); - } - - // Instantiate Logger. - // It is suggested that the user specify the class name of the containing class - // (in this case ComputeTfIdf). - private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); + EvaluationResult res = SparkPipelineRunner.create().run(pipeline); + res.close(); } - /** - * A {@link PTransform} to write, in CSV format, a mapping from term and URI - * to score. - */ - public static class WriteTfIdf - extends PTransform>>, PDone> { - - private final String output; - - public WriteTfIdf(String output) { - this.output = output; - } - - @Override - public PDone apply(PCollection>> wordToUriAndTfIdf) { - return wordToUriAndTfIdf - .apply(ParDo.named("Format").of(new DoFn>, String>() { - @Override - public void processElement(ProcessContext c) { - c.output(String.format("%s,\t%s,\t%f", - c.element().getKey(), - c.element().getValue().getKey(), - c.element().getValue().getValue())); - } - })) - .apply(TextIO.Write - .to(output) - .withSuffix(".csv")); - } - } } From 62c03373b37356e1d3c8edcfdcf0169b8be62797 Mon Sep 17 00:00:00 2001 From: Tom White Date: Sat, 20 Jun 2015 10:22:35 +0100 Subject: [PATCH 082/137] [maven-release-plugin] prepare release spark-dataflow-0.2.0 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index fb70e910b775..88f7e00e5e62 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.1.2-SNAPSHOT + 0.2.0 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.2.0 From b6c71c207c9f34375e9c5301575c4c589dd7f41b Mon Sep 17 00:00:00 2001 From: Tom White Date: Sat, 20 Jun 2015 10:22:41 +0100 Subject: [PATCH 083/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 88f7e00e5e62..171a41aa1473 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.0 + 0.2.1-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.2.0 + HEAD From 32be82eb8c76c5dfa7057528beb889ecdd9f5128 Mon Sep 17 00:00:00 2001 From: Tom White Date: Sat, 20 Jun 2015 10:28:45 +0100 Subject: [PATCH 084/137] Update README with 0.2.0 release. --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index d4de19f02c05..52f7f9b8ed24 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.1.1 + 0.2.0 and are hosted in Cloudera's repository at: From 9e6daf2bafa6a3c2dcd5f6d765aaf45719387b39 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 25 Jun 2015 12:04:00 +0100 Subject: [PATCH 085/137] Unwrap cause from SparkException if set. --- .../com/cloudera/dataflow/spark/SparkPipelineRunner.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 index 287be1d0424c..36685c348699 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -25,6 +25,7 @@ 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -114,7 +115,13 @@ public EvaluationResult run(Pipeline pipeline) { return ctxt; } catch (Exception e) { - throw new RuntimeException(e); // wrap a SparkException in a RuntimeException + // if the SparkException has a cause then wrap it in a RuntimeException + // (see https://issues.apache.org/jira/browse/SPARK-8625) + if (e instanceof SparkException && e.getCause() != null) { + throw new RuntimeException(e.getCause()); + } + // otherwise just wrap in a RuntimeException + throw new RuntimeException(e); } } From d08675cd593988925efe8be59f2e32c7623347b1 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 26 Jun 2015 13:54:34 +0100 Subject: [PATCH 086/137] Specialize CombineGlobally --- .../dataflow/spark/TransformTranslator.java | 63 ++++++++++++- .../dataflow/spark/CombineGloballyTest.java | 89 +++++++++++++++++++ 2 files changed, 151 insertions(+), 1 deletion(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.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 index ceeed1351018..0fafe7f112ca 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -17,6 +17,8 @@ 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; @@ -153,6 +155,63 @@ public void evaluate(Combine.GroupedValues transform, EvaluationConte }; } + private static final FieldGetter COMBINE_GLOBALLY_FG = new FieldGetter(Combine.Globally.class); + + private static TransformEvaluator> combineGlobally() { + return new TransformEvaluator>() { + + @Override + public void evaluate(Combine.Globally transform, EvaluationContext context) { + final Combine.CombineFn globally = COMBINE_GLOBALLY_FG.get("fn", transform); + + @SuppressWarnings("unchecked") + JavaRDDLike inRdd = (JavaRDDLike) context.getInputRDD(transform); + + final Coder iCoder = context.getInput(transform).getCoder(); + final Coder 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 inRddBytes = inRdd.map(CoderHelpers.toByteFunction(iCoder)); + + /*A*/ byte[] acc = inRddBytes.aggregate( + CoderHelpers.toByteArray(globally.createAccumulator(), aCoder), + new Function2() { + @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() { + @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 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 coder = context.getOutput(transform).getCoder(); + JavaRDD outRdd = context.getSparkContext().parallelize( + CoderHelpers.toByteArrays(ImmutableList.of(output), coder)); + context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder))); + } + }; + } + private static final FieldGetter COMBINE_PERKEY_FG = new FieldGetter(Combine.PerKey.class); private static TransformEvaluator> combinePerKey() { @@ -229,7 +288,8 @@ public Tuple2> call(KV kv) { KV kva1 = CoderHelpers.fromByteArray(acc1, kvaCoder); KV kva2 = CoderHelpers.fromByteArray(acc2, kvaCoder); VA va = keyed.mergeAccumulators(kva1.getKey(), - ImmutableList.of(kva1.getValue(), kva2.getValue())); + // don't use Guava's ImmutableList.of as values may be null + Collections.unmodifiableList(Arrays.asList(kva1.getValue(), kva2.getValue()))); return CoderHelpers.toByteArray(KV.of(kva1.getKey(), va), kvaCoder); } }); @@ -564,6 +624,7 @@ private static Map, BroadcastHelper> getSideInputs( 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.class, create()); 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 new file mode 100644 index 000000000000..be5f6dc7c2d5 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java @@ -0,0 +1,89 @@ +/* + * 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.io.Serializable; +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 WORDS = Arrays.asList(WORDS_ARRAY); + + @Test + public void test() throws Exception { + SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + PCollection 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 implements + Serializable { + + @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 accumulators) { + StringBuilder sb = new StringBuilder(); + for (StringBuilder accum : accumulators) { + if (accum != null) { + sb.append(accum.toString()); + } + } + return sb; + } + + @Override + public String extractOutput(StringBuilder accumulator) { + return accumulator.toString(); + } + + private static StringBuilder combine(StringBuilder accum, String datum) { + if (null == accum) { + return new StringBuilder(datum); + } else { + accum.append(",").append(datum); + return accum; + } + } + } +} From 80be89e4fafc23b0d78bffeb1ed99cfe48c0def5 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 26 Jun 2015 14:08:43 +0100 Subject: [PATCH 087/137] Fix bug in combinePerKey where accumulator in return value is ignored. --- .../com/cloudera/dataflow/spark/TransformTranslator.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 index 0fafe7f112ca..a7f77eb33b84 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -267,7 +267,7 @@ public Tuple2> call(KV kv) { public /*KV*/ byte[] call(/*KV*/ byte[] input) { KV kvi = CoderHelpers.fromByteArray(input, kviCoder); VA va = keyed.createAccumulator(kvi.getKey()); - keyed.addInput(kvi.getKey(), va, kvi.getValue()); + va = keyed.addInput(kvi.getKey(), va, kvi.getValue()); return CoderHelpers.toByteArray(KV.of(kvi.getKey(), va), kvaCoder); } }, @@ -277,7 +277,8 @@ public Tuple2> call(KV kv) { /*KV*/ byte[] input) { KV kva = CoderHelpers.fromByteArray(acc, kvaCoder); KV kvi = CoderHelpers.fromByteArray(input, kviCoder); - keyed.addInput(kva.getKey(), kva.getValue(), kvi.getValue()); + VA va = keyed.addInput(kva.getKey(), kva.getValue(), kvi.getValue()); + kva = KV.of(kva.getKey(), va); return CoderHelpers.toByteArray(KV.of(kva.getKey(), kva.getValue()), kvaCoder); } }, From b404b9c8219342f5501dbfff410d0874b574131f Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 26 Jun 2015 16:07:53 +0100 Subject: [PATCH 088/137] [maven-release-plugin] prepare release spark-dataflow-0.2.1 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 171a41aa1473..c624ada7e414 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.1-SNAPSHOT + 0.2.1 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.2.1 From 290193ea8f552b430da8dad7c49db09b253aa75e Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 26 Jun 2015 16:08:01 +0100 Subject: [PATCH 089/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index c624ada7e414..1ab37b7d8fe4 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.1 + 0.2.2-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.2.1 + HEAD From bacaf8c2ebb10ea441194c1399040df36405251a Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 29 Jun 2015 22:05:13 +0100 Subject: [PATCH 090/137] Factor out common code from DoFnFunction and MultiDoFnFunction. --- .../cloudera/dataflow/spark/DoFnFunction.java | 165 +-------------- .../dataflow/spark/MultiDoFnFunction.java | 74 +------ .../dataflow/spark/SparkProcessContext.java | 197 ++++++++++++++++++ .../dataflow/spark/TransformTranslator.java | 2 +- 4 files changed, 211 insertions(+), 227 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.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 index 7ae52b033f7c..c5d7ddc77f5c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -15,31 +15,14 @@ package com.cloudera.dataflow.spark; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.LinkedList; import java.util.List; 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.GlobalWindow; -import com.google.cloud.dataflow.sdk.util.TimerManager; -import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.WindowingInternals; -import com.google.cloud.dataflow.sdk.values.CodedTupleTag; -import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.spark.api.java.function.FlatMapFunction; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Dataflow's Do functions correspond to Spark's FlatMap functions. @@ -48,11 +31,6 @@ * @param Output element type. */ class DoFnFunction implements FlatMapFunction, O> { - private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); - - private static final Collection GLOBAL_WINDOWS = - Collections.singletonList(GlobalWindow.INSTANCE); - private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -70,10 +48,9 @@ class DoFnFunction implements FlatMapFunction, O> { this.mSideInputs = sideInputs; } - @Override public Iterable call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(mFunction); + ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); //setup mFunction.startBundle(ctxt); ctxt.setup(); @@ -87,150 +64,18 @@ public Iterable call(Iterator iter) throws Exception { return ctxt.outputs; } - private class ProcCtxt extends DoFn.ProcessContext { + private class ProcCtxt extends SparkProcessContext { private final List outputs = new LinkedList<>(); - private I element; - - ProcCtxt(DoFn fn) { - fn.super(); - } - - void setup() { - super.setupDelegateAggregators(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return mRuntimeContext.getPipelineOptions(); - } - @Override - public T sideInput(PCollectionView view) { - @SuppressWarnings("unchecked") - BroadcastHelper>> broadcastHelper = - (BroadcastHelper>>) mSideInputs.get(view.getTagInternal()); - Iterable> contents = broadcastHelper.getValue(); - return view.fromIterableInternal(contents); + ProcCtxt(DoFn fn, SparkRuntimeContext runtimeContext, Map, + BroadcastHelper> sideInputs) { + super(fn, runtimeContext, sideInputs); } @Override public synchronized void output(O o) { outputs.add(o); } - - @Override - public void sideOutput(TupleTag 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 void sideOutputWithTimestamp(TupleTag 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 Aggregator createAggregatorInternal( - String named, - Combine.CombineFn combineFn) { - return mRuntimeContext.createAggregator(named, combineFn); - } - - @Override - public I element() { - return element; - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } - - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - - @Override - public Collection windows() { - return GLOBAL_WINDOWS; - } - - @Override - public void outputWindowedValue(O output, Instant timestamp, Collection windows) { - output(output); - } - - @Override - public KeyedState keyedState() { - throw new UnsupportedOperationException( - "WindowingInternals#keyedState() is not yet supported."); - - } - - @Override - public void store(CodedTupleTag tag, T value, Instant timestamp) - throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#store() is not yet supported."); - } - - @Override - public void writeToTagList(CodedTupleTag tag, T value) throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#writeToTagList() is not yet supported."); - } - - @Override - public void deleteTagList(CodedTupleTag tag) { - throw new UnsupportedOperationException( - "WindowingInternals#deleteTagList() is not yet supported."); - } - - @Override - public Iterable readTagList(CodedTupleTag tag) throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#readTagList() is not yet supported."); - } - - @Override - public Map, Iterable> readTagList(List> tags) - throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#readTagList() is not yet supported."); - } - - @Override - public TimerManager getTimerManager() { - throw new UnsupportedOperationException( - "WindowingInternals#getTimerManager() is not yet supported."); - } - - @Override - public void writePCollectionViewData(TupleTag tag, - Iterable> data, Coder elemCoder) throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#writePCollectionViewData() is not yet supported."); - } - }; - } - } } 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 index 243fa67a6b68..e6cb664e3b8c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -18,14 +18,7 @@ import java.util.Iterator; import java.util.Map; -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.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.WindowingInternals; -import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -43,10 +36,6 @@ * @param Output type for DoFunction. */ class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { - // TODO: I think implementing decoding logic will allow us to do away with having two types of - // DoFunctions. Josh originally made these two classes in order to help ease the typing of - // results. Correctly using coders should just fix this. - private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final TupleTag mMainOutputTag; @@ -65,13 +54,16 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT @Override public Iterable, Object>> call(Iterator iter) throws Exception { - ProcCtxt ctxt = new ProcCtxt(mFunction); + ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); + //setup mFunction.startBundle(ctxt); ctxt.setup(); + //operation while (iter.hasNext()) { ctxt.element = iter.next(); mFunction.processElement(ctxt); } + //cleanup mFunction.finishBundle(ctxt); return Iterables.transform(ctxt.outputs.entries(), new Function, Object>, Tuple2, Object>>() { @@ -82,31 +74,13 @@ public Tuple2, Object> apply(Map.Entry, Object> input) { }); } - private class ProcCtxt extends DoFn.ProcessContext { + private class ProcCtxt extends SparkProcessContext { private final Multimap, Object> outputs = LinkedListMultimap.create(); - private I element; - - ProcCtxt(DoFn fn) { - fn.super(); - } - - void setup() { - super.setupDelegateAggregators(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return mRuntimeContext.getPipelineOptions(); - } - @Override - public T sideInput(PCollectionView view) { - @SuppressWarnings("unchecked") - BroadcastHelper>> broadcastHelper = - (BroadcastHelper>>) mSideInputs.get(view.getTagInternal()); - Iterable> contents = broadcastHelper.getValue(); - return view.fromIterableInternal(contents); + ProcCtxt(DoFn fn, SparkRuntimeContext runtimeContext, Map, + BroadcastHelper> sideInputs) { + super(fn, runtimeContext, sideInputs); } @Override @@ -123,37 +97,5 @@ public synchronized void sideOutput(TupleTag tag, T t) { public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { outputs.put(tupleTag, t); } - - @Override - public Aggregator createAggregatorInternal( - String named, - Combine.CombineFn combineFn) { - return mRuntimeContext.createAggregator(named, combineFn); - } - - @Override - public I element() { - return element; - } - - @Override - public void outputWithTimestamp(O output, Instant timestamp) { - output(output); - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public BoundedWindow window() { - return null; - } - - @Override - public WindowingInternals windowingInternals() { - return null; - } } } 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 new file mode 100644 index 000000000000..fd4e04828b8d --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -0,0 +1,197 @@ +/* + * 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.Collections; +import java.util.List; +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.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.TimerManager; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class SparkProcessContext extends DoFn.ProcessContext { + + private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); + + private static final Collection GLOBAL_WINDOWS = + Collections.singletonList(GlobalWindow.INSTANCE); + + private final DoFn mFunction; + private final SparkRuntimeContext mRuntimeContext; + private final Map, BroadcastHelper> mSideInputs; + + protected I element; + + SparkProcessContext(DoFn fn, + SparkRuntimeContext runtime, + Map, BroadcastHelper> sideInputs) { + fn.super(); + this.mFunction = fn; + this.mRuntimeContext = runtime; + this.mSideInputs = sideInputs; + } + + void setup() { + super.setupDelegateAggregators(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return mRuntimeContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + @SuppressWarnings("unchecked") + BroadcastHelper>> broadcastHelper = + (BroadcastHelper>>) mSideInputs.get(view.getTagInternal()); + Iterable> contents = broadcastHelper.getValue(); + return view.fromIterableInternal(contents); + } + + @Override + public abstract void output(O output); + + @Override + public void sideOutput(TupleTag 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 void sideOutputWithTimestamp(TupleTag 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 Aggregator createAggregatorInternal( + String named, + Combine.CombineFn combineFn) { + return mRuntimeContext.createAggregator(named, combineFn); + } + + @Override + public I element() { + return element; + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + output(output); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public WindowingInternals windowingInternals() { + return new WindowingInternals() { + + @Override + public Collection windows() { + return GLOBAL_WINDOWS; + } + + @Override + public void outputWindowedValue(O output, Instant timestamp, Collection windows) { + output(output); + } + + @Override + public KeyedState keyedState() { + throw new UnsupportedOperationException( + "WindowingInternals#keyedState() is not yet supported."); + + } + + @Override + public void store(CodedTupleTag tag, T value, Instant timestamp) + throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#store() is not yet supported."); + } + + @Override + public void writeToTagList(CodedTupleTag tag, T value) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#writeToTagList() is not yet supported."); + } + + @Override + public void deleteTagList(CodedTupleTag tag) { + throw new UnsupportedOperationException( + "WindowingInternals#deleteTagList() is not yet supported."); + } + + @Override + public Iterable readTagList(CodedTupleTag tag) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#readTagList() is not yet supported."); + } + + @Override + public Map, Iterable> readTagList(List> tags) + throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#readTagList() is not yet supported."); + } + + @Override + public TimerManager getTimerManager() { + throw new UnsupportedOperationException( + "WindowingInternals#getTimerManager() is not yet supported."); + } + + @Override + public void writePCollectionViewData(TupleTag tag, + Iterable> data, Coder elemCoder) throws IOException { + throw new UnsupportedOperationException( + "WindowingInternals#writePCollectionViewData() is not yet supported."); + } + }; + } + +} 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 index a7f77eb33b84..6b78d9e23f77 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -539,7 +539,7 @@ private static TransformEvaluator> viewAsSingleton() { return new TransformEvaluator>() { @Override public void evaluate(View.AsSingleton transform, EvaluationContext context) { - //PROBABLY INCORRECT. Fix it. + // TODO: PROBABLY INCORRECT. Fix it. Iterable input = context.get(context.getInput(transform)); context.setPView(context.getOutput(transform), Iterables.transform(input, new WindowingFunction())); From 13edbec10e11987c5b5ad3d5bc0a68b7914391ee Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 29 Jun 2015 22:27:55 +0100 Subject: [PATCH 091/137] Implement Aggregator#getCombineFn. --- .../dataflow/spark/SparkRuntimeContext.java | 32 ++---------- .../spark/aggregators/NamedAggregators.java | 50 +++---------------- 2 files changed, 10 insertions(+), 72 deletions(-) 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 index 35e3937d2278..fbc16d65dac2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -31,7 +31,6 @@ 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.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import org.apache.spark.Accumulator; @@ -95,30 +94,6 @@ public synchronized PipelineOptions getPipelineOptions() { return deserializePipelineOptions(serializedPipelineOptions); } - /** - * Creates and aggregator and associates it with the specified name. - * - * @param named Name of aggregator. - * @param sfunc Serializable function used in aggregation. - * @param Type of inputs to aggregator. - * @param Type of aggregator outputs. - * @return Specified aggregator - */ - public synchronized Aggregator createAggregator( - String named, - SerializableFunction, OUT> sfunc) { - @SuppressWarnings("unchecked") - Aggregator aggregator = (Aggregator) aggregators.get(named); - if (aggregator == null) { - NamedAggregators.SerFunctionState state = new NamedAggregators - .SerFunctionState<>(sfunc); - accum.add(new NamedAggregators(named, state)); - aggregator = new SparkAggregator<>(named, state); - aggregators.put(named, aggregator); - } - return aggregator; - } - /** * Creates and aggregator and associates it with the specified name. * @@ -192,9 +167,9 @@ private Coder getCoder(Combine.CombineFn combiner) { */ private static class SparkAggregator implements Aggregator, Serializable { private final String name; - private final NamedAggregators.State state; + private final NamedAggregators.State state; - SparkAggregator(String name, NamedAggregators.State state) { + SparkAggregator(String name, NamedAggregators.State state) { this.name = name; this.state = state; } @@ -211,8 +186,7 @@ public void addValue(IN elem) { @Override public Combine.CombineFn getCombineFn() { - //TODO: Support this. - throw new UnsupportedOperationException("getCombineFn is not yet supported."); + return state.getCombineFn(); } } } 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 index 418e684110ba..d51e40449104 100644 --- 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 @@ -25,7 +25,6 @@ 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.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.common.collect.ImmutableList; import com.cloudera.dataflow.spark.SparkRuntimeContext; @@ -124,6 +123,8 @@ public interface State extends Serializable { INTER current(); OUT render(); + + Combine.CombineFn getCombineFn(); } /** @@ -167,6 +168,11 @@ public OUT render() { return combineFn.extractOutput(state); } + @Override + public Combine.CombineFn getCombineFn() { + return combineFn; + } + private void writeObject(ObjectOutputStream oos) throws IOException { oos.writeObject(ctxt); oos.writeObject(combineFn); @@ -193,46 +199,4 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound } } - /** - * states correspond to dataflow objects. this one => serializable function - */ - public static class SerFunctionState implements State { - - private final SerializableFunction, OUT> sfunc; - private OUT state; - - public SerFunctionState(SerializableFunction, OUT> sfunc) { - this.sfunc = sfunc; - this.state = sfunc.apply(ImmutableList.of()); - } - - @Override - public void update(IN element) { - @SuppressWarnings("unchecked") - IN thisState = (IN) state; - this.state = sfunc.apply(ImmutableList.of(element, thisState)); - } - - @Override - public State merge(State other) { - // Add exception catching and logging here. - @SuppressWarnings("unchecked") - IN thisState = (IN) state; - @SuppressWarnings("unchecked") - IN otherCurrent = (IN) other.current(); - this.state = sfunc.apply(ImmutableList.of(thisState, otherCurrent)); - return this; - } - - @Override - public OUT current() { - return state; - } - - @Override - public OUT render() { - return state; - } - } - } From 89e2bb521d8ed8480a2af102614248f29942cbe2 Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 29 Jun 2015 22:59:42 +0100 Subject: [PATCH 092/137] Implement getAggregatorValues. --- .../dataflow/spark/EvaluationContext.java | 3 +-- .../dataflow/spark/SparkRuntimeContext.java | 19 +++++++++++++++++++ .../spark/MultiOutputWordCountTest.java | 17 +++++++++++++++-- 3 files changed, 35 insertions(+), 4 deletions(-) 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 index c7aa7c6366fb..df3f7f7fa98c 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -168,8 +168,7 @@ public T getAggregatorValue(String named, Class resultType) { @Override public AggregatorValues getAggregatorValues(Aggregator aggregator) throws AggregatorRetrievalException { - //TODO: Support this. - throw new UnsupportedOperationException("getAggregatorValues is not yet supported."); + return runtime.getAggregatorValues(aggregator); } @Override 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 index fbc16d65dac2..51db39b8b9e7 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -17,6 +17,7 @@ import java.io.IOException; import java.io.Serializable; +import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -27,12 +28,14 @@ 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; @@ -90,6 +93,22 @@ public T getAggregatorValue(String aggregatorName, Class typeClass) { return accum.value().getValue(aggregatorName, typeClass); } + public AggregatorValues getAggregatorValues(Aggregator aggregator) { + final T aggregatorValue = (T) getAggregatorValue(aggregator.getName(), + aggregator.getCombineFn().getOutputType().getRawType()); + return new AggregatorValues() { + @Override + public Collection getValues() { + return ImmutableList.of(aggregatorValue); + } + + @Override + public Map getValuesAtSteps() { + throw new UnsupportedOperationException("getValuesAtSteps is not supported."); + } + }; + } + public synchronized PipelineOptions getPipelineOptions() { return deserializePipelineOptions(serializedPipelineOptions); } 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 index b16320d11fba..bf2ecdcad742 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -18,6 +18,7 @@ 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; @@ -36,6 +37,7 @@ 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; @@ -56,7 +58,8 @@ public void testRun() throws Exception { PCollection union = list.apply(Flatten.pCollections()); PCollectionView regexView = regex.apply(View.asSingleton()); - PCollectionTuple luc = union.apply(new CountWords(regexView)); + CountWords countWords = new CountWords(regexView); + PCollectionTuple luc = union.apply(countWords); PCollection unique = luc.get(lowerCnts).apply( ApproximateUnique.>globally(16)); @@ -70,6 +73,10 @@ public void testRun() throws Exception { Assert.assertEquals(18, actualTotalWords); int actualMaxWordLength = res.getAggregatorValue("maxWordLength", Integer.class); Assert.assertEquals(6, actualMaxWordLength); + AggregatorValues aggregatorValues = res.getAggregatorValues(countWords + .getTotalWordsAggregator()); + Assert.assertEquals(18, Iterables.getOnlyElement(aggregatorValues.getValues()).intValue()); + res.close(); } @@ -108,16 +115,18 @@ public void processElement(ProcessContext c) { public static class CountWords extends PTransform, PCollectionTuple> { private final PCollectionView regex; + private final ExtractWordsFn extractWordsFn; public CountWords(PCollectionView regex) { this.regex = regex; + this.extractWordsFn = new ExtractWordsFn(regex); } @Override public PCollectionTuple apply(PCollection lines) { // Convert lines of text into individual words. PCollectionTuple lowerUpper = lines - .apply(ParDo.of(new ExtractWordsFn(regex)) + .apply(ParDo.of(extractWordsFn) .withSideInputs(regex) .withOutputTags(lower, TupleTagList.of(upper))); lowerUpper.get(lower).setCoder(StringUtf8Coder.of()); @@ -130,5 +139,9 @@ public PCollectionTuple apply(PCollection lines) { .of(lowerCnts, lowerCounts) .and(upperCnts, upperCounts); } + + Aggregator getTotalWordsAggregator() { + return extractWordsFn.totalWords; + } } } From 78d66145ed37ef82bb8cc8f1f7a4783d018cceae Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 30 Jun 2015 09:00:04 +0100 Subject: [PATCH 093/137] More cleanup. View.AsSingleton is already exercised by the TfIdf test. --- .../java/com/cloudera/dataflow/spark/EvaluationContext.java | 3 ++- .../java/com/cloudera/dataflow/spark/SparkContextFactory.java | 2 +- .../java/com/cloudera/dataflow/spark/SparkProcessContext.java | 4 +--- .../java/com/cloudera/dataflow/spark/TransformTranslator.java | 2 -- 4 files changed, 4 insertions(+), 7 deletions(-) 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 index df3f7f7fa98c..53372645ea61 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -193,8 +193,9 @@ public void close() { SparkContextFactory.stopSparkContext(jsc); } + /** The runner is blocking. */ @Override public State getState() { - return State.UNKNOWN; + return State.DONE; } } 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 index 483899a156f3..b7570b39e233 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java @@ -59,7 +59,7 @@ public static synchronized void stopSparkContext(JavaSparkContext context) { private static JavaSparkContext createSparkContext(String master) { SparkConf conf = new SparkConf(); conf.setMaster(master); - conf.setAppName("spark pipeline job"); + conf.setAppName("spark dataflow pipeline job"); conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); return new JavaSparkContext(conf); } 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 index fd4e04828b8d..12fb4e043897 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -40,12 +40,11 @@ abstract class SparkProcessContext extends DoFn.ProcessContext { - private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); + private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class); private static final Collection GLOBAL_WINDOWS = Collections.singletonList(GlobalWindow.INSTANCE); - private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -55,7 +54,6 @@ abstract class SparkProcessContext extends DoFn.ProcessContext { SparkRuntimeContext runtime, Map, BroadcastHelper> sideInputs) { fn.super(); - this.mFunction = fn; this.mRuntimeContext = runtime; this.mSideInputs = sideInputs; } 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 index 6b78d9e23f77..26894241412b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -539,7 +539,6 @@ private static TransformEvaluator> viewAsSingleton() { return new TransformEvaluator>() { @Override public void evaluate(View.AsSingleton transform, EvaluationContext context) { - // TODO: PROBABLY INCORRECT. Fix it. Iterable input = context.get(context.getInput(transform)); context.setPView(context.getOutput(transform), Iterables.transform(input, new WindowingFunction())); @@ -552,7 +551,6 @@ private static TransformEvaluator> viewAsIter() { @Override public void evaluate(View.AsIterable transform, EvaluationContext context) { Iterable input = context.get(context.getInput(transform)); - context.setPView(context.getOutput(transform), Iterables.transform(input, new WindowingFunction())); } From 5069eedb6d3faa6560a79c82f2d091ff4e2becc0 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 30 Jun 2015 09:28:40 +0100 Subject: [PATCH 094/137] Set the RDD's name from the PValue's name, to help diagnosis. --- .../java/com/cloudera/dataflow/spark/EvaluationContext.java | 5 +++++ 1 file changed, 5 insertions(+) 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 index 53372645ea61..7906775211f2 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -118,6 +118,11 @@ void setPView(PValue view, Iterable> value) { } void setRDD(PValue pvalue, JavaRDDLike rdd) { + try { + rdd.rdd().setName(pvalue.getName()); + } catch (IllegalStateException e) { + // name not set, ignore + } rdds.put(pvalue, rdd); leafRdds.add(rdd); } From 2820534a54fe77bf8220da9bb0cd1d4ce6c8b2b3 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 3 Jul 2015 18:38:02 +0100 Subject: [PATCH 095/137] Resolve some generics warnings with some fancier footwork --- runners/spark/README.md | 2 +- runners/spark/pom.xml | 2 +- .../com/cloudera/dataflow/hadoop/HadoopIO.java | 17 +++++++++-------- .../dataflow/spark/EvaluationContext.java | 2 +- .../dataflow/spark/SparkPipelineRunner.java | 13 +++++++------ .../dataflow/spark/SparkRuntimeContext.java | 7 ++++--- .../dataflow/spark/TransformEvaluator.java | 2 +- .../dataflow/spark/TransformTranslator.java | 6 +++--- .../spark/HadoopFileFormatPipelineTest.java | 15 ++++++++++----- 9 files changed, 37 insertions(+), 29 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 52f7f9b8ed24..d93f55470383 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -66,7 +66,7 @@ First download a text document to use as input: Then run the [word count example][wc] from the SDK using a single threaded Spark instance in local mode: - mvn exec:exec -Dclass=com.google.cloud.dataflow.examples.WordCount \ + mvn exec:exec -DmainClass=com.google.cloud.dataflow.examples.WordCount \ -Dinput=/tmp/kinglear.txt -Doutput=/tmp/out -Drunner=SparkPipelineRunner \ -DsparkMaster=local diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 1ab37b7d8fe4..03821082aa10 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -194,7 +194,7 @@ License. -classpath - ${class} + ${mainClass} --input=${input} --output=${output} --runner=${runner} 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 index 434a1326b663..5873b9f4c403 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -32,20 +32,21 @@ public static final class Read { private Read() { } - public static Bound from(String filepattern) { - return new Bound().from(filepattern); + public static Bound from(String filepattern) { + return new Bound().from(filepattern); } - public static Bound withFormatClass(Class> format) { - return new Bound().withFormatClass(format); + public static Bound withFormatClass( + Class> format) { + return new Bound().withFormatClass(format); } - public static Bound withKeyClass(Class key) { - return new Bound().withKeyClass(key); + public static Bound withKeyClass(Class key) { + return new Bound().withKeyClass(key); } - public static Bound withValueClass(Class value) { - return new Bound().withValueClass(value); + public static Bound withValueClass(Class value) { + return new Bound().withValueClass(value); } public static class Bound extends PTransform>> { 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 index 7906775211f2..56f85213b9c0 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -127,7 +127,7 @@ void setRDD(PValue pvalue, JavaRDDLike rdd) { leafRdds.add(rdd); } - JavaRDDLike getInputRDD(PTransform transform) { + JavaRDDLike getInputRDD(PTransform transform) { return getRDD((PValue) getInput(transform)); } 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 index 36685c348699..792888dbeac5 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -183,15 +183,16 @@ public void visitTransform(TransformTreeNode node) { doVisitTransform(node); } - private void doVisitTransform(TransformTreeNode node) { + private > + void doVisitTransform(TransformTreeNode node) { + @SuppressWarnings("unchecked") PT transform = (PT) node.getTransform(); @SuppressWarnings("unchecked") - TransformEvaluator evaluator = (TransformEvaluator) - TransformTranslator.getTransformEvaluator(transform.getClass()); + Class transformClass = (Class) (Class) transform.getClass(); + TransformEvaluator evaluator = TransformTranslator.getTransformEvaluator(transformClass); LOG.info("Evaluating {}", transform); - AppliedPTransform appliedTransform = - AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), - (PTransform) transform); + AppliedPTransform appliedTransform = + AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform); ctxt.setCurrentTransform(appliedTransform); evaluator.evaluate(transform, ctxt); ctxt.setCurrentTransform(null); 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 index 51db39b8b9e7..ec590a96fa18 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkRuntimeContext.java @@ -94,8 +94,9 @@ public T getAggregatorValue(String aggregatorName, Class typeClass) { } public AggregatorValues getAggregatorValues(Aggregator aggregator) { - final T aggregatorValue = (T) getAggregatorValue(aggregator.getName(), - aggregator.getCombineFn().getOutputType().getRawType()); + @SuppressWarnings("unchecked") + Class aggValueClass = (Class) aggregator.getCombineFn().getOutputType().getRawType(); + final T aggregatorValue = getAggregatorValue(aggregator.getName(), aggValueClass); return new AggregatorValues() { @Override public Collection getValues() { @@ -150,7 +151,7 @@ public CoderRegistry getCoderRegistry() { return coderRegistry; } - private Coder getCoder(Combine.CombineFn combiner) { + private Coder getCoder(Combine.CombineFn combiner) { try { if (combiner.getClass() == Sum.SumIntegerFn.class) { return getCoderRegistry().getDefaultCoder(TypeDescriptor.of(Integer.class)); 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 index bf45d129b966..52842d575ede 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformEvaluator.java @@ -19,6 +19,6 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; -public interface TransformEvaluator extends Serializable { +public interface TransformEvaluator> extends Serializable { void evaluate(PT transform, EvaluationContext context); } 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 index 26894241412b..b0fd4a3b62ea 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -633,12 +633,12 @@ private static Map, BroadcastHelper> getSideInputs( EVALUATORS.put(Window.Bound.class, window()); } - public static boolean hasTransformEvaluator(Class clazz) { + public static > boolean hasTransformEvaluator(Class clazz) { return EVALUATORS.containsKey(clazz); } - public static TransformEvaluator getTransformEvaluator(Class - clazz) { + public static > TransformEvaluator getTransformEvaluator(Class + clazz) { @SuppressWarnings("unchecked") TransformEvaluator transform = (TransformEvaluator) EVALUATORS.get(clazz); if (transform == null) { 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 index 9aa634e444ca..1fd8e4115ce7 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -35,6 +35,7 @@ import org.apache.hadoop.io.SequenceFile; 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.junit.Before; import org.junit.Rule; @@ -63,11 +64,15 @@ public void testGeneric() throws Exception { populateFile(); Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - PCollection> input = (PCollection>) - p.apply(HadoopIO.Read.from(inputFile.getAbsolutePath()) - .withFormatClass(SequenceFileInputFormat.class) - .withKeyClass(IntWritable.class) - .withValueClass(Text.class)); + @SuppressWarnings("unchecked") + Class> inputFormatClass = + (Class>) (Class) SequenceFileInputFormat.class; + HadoopIO.Read.Bound bound = + HadoopIO.Read.from(inputFile.getAbsolutePath()) + .withKeyClass(IntWritable.class) + .withValueClass(Text.class) + .withFormatClass(inputFormatClass); + PCollection> input = p.apply(bound); input.apply(ParDo.of(new TabSeparatedString())) .apply(TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding()); EvaluationResult res = SparkPipelineRunner.create().run(p); From b2f495ecb98f4cd67c6676682f848af717e2aeb4 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 6 Jul 2015 12:23:07 +0100 Subject: [PATCH 096/137] Remove some HadoopIO.Read.Bound factory methods and fluent setters; always set key/value at creation --- .../cloudera/dataflow/hadoop/HadoopIO.java | 29 ++----------------- .../spark/HadoopFileFormatPipelineTest.java | 5 ++-- 2 files changed, 4 insertions(+), 30 deletions(-) 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 index 5873b9f4c403..803c49556302 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -32,21 +32,8 @@ public static final class Read { private Read() { } - public static Bound from(String filepattern) { - return new Bound().from(filepattern); - } - - public static Bound withFormatClass( - Class> format) { - return new Bound().withFormatClass(format); - } - - public static Bound withKeyClass(Class key) { - return new Bound().withKeyClass(key); - } - - public static Bound withValueClass(Class value) { - return new Bound().withValueClass(value); + public static Bound withKeyValueClass(Class key, Class value) { + return new Bound<>(null, null, key, value); } public static class Bound extends PTransform>> { @@ -56,10 +43,6 @@ public static class Bound extends PTransform> private final Class keyClass; private final Class valueClass; - Bound() { - this(null, null, null, null); - } - Bound(String filepattern, Class> format, Class key, Class value) { this.filepattern = filepattern; @@ -76,14 +59,6 @@ public Bound withFormatClass(Class> format return new Bound<>(filepattern, format, keyClass, valueClass); } - public Bound withKeyClass(Class key) { - return new Bound<>(filepattern, formatClass, key, valueClass); - } - - public Bound withValueClass(Class value) { - return new Bound<>(filepattern, formatClass, keyClass, value); - } - public String getFilepattern() { return filepattern; } 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 index 1fd8e4115ce7..127d58f5f2f8 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -68,9 +68,8 @@ public void testGeneric() throws Exception { Class> inputFormatClass = (Class>) (Class) SequenceFileInputFormat.class; HadoopIO.Read.Bound bound = - HadoopIO.Read.from(inputFile.getAbsolutePath()) - .withKeyClass(IntWritable.class) - .withValueClass(Text.class) + HadoopIO.Read.withKeyValueClass(IntWritable.class, Text.class). + from(inputFile.getAbsolutePath()) .withFormatClass(inputFormatClass); PCollection> input = p.apply(bound); input.apply(ParDo.of(new TabSeparatedString())) From b47a8d0a31f8f160a26cd5b41b3317857969f66a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 6 Jul 2015 12:23:07 +0100 Subject: [PATCH 097/137] Remove some HadoopIO.Read.Bound factory methods and fluent setters; always set key/value at creation --- .../cloudera/dataflow/hadoop/HadoopIO.java | 30 +++++++------------ .../spark/HadoopFileFormatPipelineTest.java | 4 +-- 2 files changed, 12 insertions(+), 22 deletions(-) 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 index 803c49556302..587e66e42a37 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -32,8 +32,9 @@ public static final class Read { private Read() { } - public static Bound withKeyValueClass(Class key, Class value) { - return new Bound<>(null, null, key, value); + public static Bound from(String filepattern, Class> format, + Class key, Class value) { + return new Bound<>(filepattern, format, key, value); } public static class Bound extends PTransform>> { @@ -45,20 +46,20 @@ public static class Bound extends PTransform> Bound(String filepattern, Class> format, Class key, Class 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 Bound from(String file) { - return new Bound<>(file, formatClass, keyClass, valueClass); - } - - public Bound withFormatClass(Class> format) { - return new Bound<>(filepattern, format, keyClass, valueClass); - } - public String getFilepattern() { return filepattern; } @@ -77,15 +78,6 @@ public Class getKeyClass() { @Override public PCollection> apply(PInput input) { - Preconditions.checkNotNull(filepattern, - "need to set the filepattern of an HadoopIO.Read transform"); - Preconditions.checkNotNull(formatClass, - "need to set the format class of an HadoopIO.Read transform"); - Preconditions.checkNotNull(keyClass, - "need to set the key class of an HadoopIO.Read transform"); - Preconditions.checkNotNull(valueClass, - "need to set the value class of an HadoopIO.Read transform"); - return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); } 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 index 127d58f5f2f8..ba6f7b0d1be5 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -68,9 +68,7 @@ public void testGeneric() throws Exception { Class> inputFormatClass = (Class>) (Class) SequenceFileInputFormat.class; HadoopIO.Read.Bound bound = - HadoopIO.Read.withKeyValueClass(IntWritable.class, Text.class). - from(inputFile.getAbsolutePath()) - .withFormatClass(inputFormatClass); + HadoopIO.Read.from(inputFile.getAbsolutePath(), inputFormatClass, IntWritable.class, Text.class); PCollection> input = p.apply(bound); input.apply(ParDo.of(new TabSeparatedString())) .apply(TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding()); From c51bc326baaefe1aeb0c9373cf8cc767a8ee5229 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 7 Jul 2015 13:37:40 +0100 Subject: [PATCH 098/137] Fix checkstyle error --- .../src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 index 587e66e42a37..533dd3065894 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -32,8 +32,8 @@ public static final class Read { private Read() { } - public static Bound from(String filepattern, Class> format, - Class key, Class value) { + public static Bound from(String filepattern, + Class> format, Class key, Class value) { return new Bound<>(filepattern, format, key, value); } From 7cff30498d0bcc0bdafddc0f0a6190d7b28d56a6 Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 8 Jul 2015 20:57:25 +0100 Subject: [PATCH 099/137] Delay converting PCollection values to bytes in case they are only used for views. --- .../dataflow/spark/EvaluationContext.java | 90 ++++++++++++++----- .../dataflow/spark/TransformTranslator.java | 4 +- 2 files changed, 68 insertions(+), 26 deletions(-) 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 index 56f85213b9c0..649cbe9088e3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -51,8 +51,8 @@ public class EvaluationContext implements EvaluationResult { private final Pipeline pipeline; private final SparkRuntimeContext runtime; private final CoderRegistry registry; - private final Map> rdds = new LinkedHashMap<>(); - private final Set> leafRdds = new LinkedHashSet<>(); + private final Map> pcollections = new LinkedHashMap<>(); + private final Set> leafRdds = new LinkedHashSet<>(); private final Set multireads = new LinkedHashSet<>(); private final Map pobjects = new LinkedHashMap<>(); private final Map>> pview = new LinkedHashMap<>(); @@ -65,6 +65,52 @@ public EvaluationContext(JavaSparkContext jsc, 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 { + + private Iterable values; + private Coder coder; + private JavaRDDLike rdd; + + public RDDHolder(Iterable values, Coder coder) { + this.values = values; + this.coder = coder; + } + + public RDDHolder(JavaRDDLike rdd) { + this.rdd = rdd; + } + + public JavaRDDLike getRDD() { + if (rdd == null) { + rdd = jsc.parallelize(CoderHelpers.toByteArrays(values, coder)) + .map(CoderHelpers.fromByteFunction(coder)); + } + return rdd; + } + + public Iterable getValues(PCollection pcollection) { + if (values == null) { + coder = pcollection.getCoder(); + JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); + List clientBytes = bytesRDD.collect(); + values = Iterables.transform(clientBytes, new Function() { + @Override + public T apply(byte[] bytes) { + return CoderHelpers.fromByteArray(bytes, coder); + } + }); + } + return values; + } + } + JavaSparkContext getSparkContext() { return jsc; } @@ -97,17 +143,23 @@ O getOutput(PTransform transform) { return output; } - void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + void setOutputRDD(PTransform transform, JavaRDDLike rdd) { setRDD((PValue) getOutput(transform), rdd); } + void setOutputRDDFromValues(PTransform transform, Iterable values, + Coder coder) { + pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder)); + } + void setPView(PValue view, Iterable> value) { pview.put(view, value); } JavaRDDLike getRDD(PValue pvalue) { - JavaRDDLike rdd = rdds.get(pvalue); - leafRdds.remove(rdd); + 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(); @@ -117,14 +169,15 @@ void setPView(PValue view, Iterable> value) { return rdd; } - void setRDD(PValue pvalue, JavaRDDLike rdd) { + void setRDD(PValue pvalue, JavaRDDLike rdd) { try { rdd.rdd().setName(pvalue.getName()); } catch (IllegalStateException e) { // name not set, ignore } - rdds.put(pvalue, rdd); - leafRdds.add(rdd); + RDDHolder rddHolder = new RDDHolder<>(rdd); + pcollections.put(pvalue, rddHolder); + leafRdds.add(rddHolder); } JavaRDDLike getInputRDD(PTransform transform) { @@ -142,7 +195,8 @@ Iterable> getPCollectionView(PCollectionView view) { * effects). */ void computeOutputs() { - for (JavaRDDLike rdd : leafRdds) { + 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 } @@ -155,8 +209,8 @@ public T get(PValue value) { T result = (T) pobjects.get(value); return result; } - if (rdds.containsKey(value)) { - JavaRDDLike rdd = rdds.get(value); + if (pcollections.containsKey(value)) { + JavaRDDLike rdd = pcollections.get(value).getRDD(); @SuppressWarnings("unchecked") T res = (T) Iterables.getOnlyElement(rdd.collect()); pobjects.put(value, res); @@ -179,18 +233,8 @@ public AggregatorValues getAggregatorValues(Aggregator aggregator) @Override public Iterable get(PCollection pcollection) { @SuppressWarnings("unchecked") - JavaRDDLike rdd = (JavaRDDLike) getRDD(pcollection); - // Use a coder to convert the objects in the PCollection to byte arrays, so they - // can be transferred over the network. - final Coder coder = pcollection.getCoder(); - JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); - List clientBytes = bytesRDD.collect(); - return Iterables.transform(clientBytes, new Function() { - @Override - public T apply(byte[] bytes) { - return CoderHelpers.fromByteArray(bytes, coder); - } - }); + RDDHolder rddHolder = (RDDHolder) pcollections.get(pcollection); + return rddHolder.getValues(pcollection); } @Override 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 index b0fd4a3b62ea..195766e9dc02 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -528,9 +528,7 @@ public void evaluate(Create transform, EvaluationContext context) { // Use a coder to convert the objects in the PCollection to byte arrays, so they // can be transferred over the network. Coder coder = context.getOutput(transform).getCoder(); - JavaRDD rdd = context.getSparkContext().parallelize( - CoderHelpers.toByteArrays(elems, coder)); - context.setOutputRDD(transform, rdd.map(CoderHelpers.fromByteFunction(coder))); + context.setOutputRDDFromValues(transform, elems, coder); } }; } From 2d00b3bd7c55ebde5ed3242b697749b111dfb9ce Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 9 Jul 2015 10:40:18 +0100 Subject: [PATCH 100/137] Add a system property, dataflow.spark.directBroadcast, to allow pipelines to bypass coders for broadcasts. --- .../dataflow/spark/BroadcastHelper.java | 102 +++++++++++++----- .../dataflow/spark/TransformTranslator.java | 2 +- 2 files changed, 79 insertions(+), 25 deletions(-) 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 index 6a26787f320e..2622ce934d8d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -25,39 +25,93 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class BroadcastHelper implements Serializable { +abstract class BroadcastHelper implements Serializable { - private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); + /** + * If the property dataflow.spark.directBroadcast is set to + * 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 com.google.cloud.dataflow.sdk.coders.Coder}. + */ + public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast"; - private Broadcast bcast; - private final Coder coder; - private transient T value; + private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); - BroadcastHelper(T value, Coder coder) { - this.value = value; - this.coder = coder; + public static BroadcastHelper create(T value, Coder coder) { + if (Boolean.getBoolean(DIRECT_BROADCAST)) { + return new DirectBroadcastHelper<>(value); + } + return new CodedBroadcastHelper<>(value, coder); } - public synchronized T getValue() { - if (value == null) { - value = deserialize(); + 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 + */ + static class DirectBroadcastHelper extends BroadcastHelper { + private Broadcast bcast; + private transient T value; + + DirectBroadcastHelper(T value) { + this.value = value; + } + + public synchronized T getValue() { + if (value == null) { + value = bcast.getValue(); + } + return value; } - return value; - } - public void broadcast(JavaSparkContext jsc) { - this.bcast = jsc.broadcast(CoderHelpers.toByteArray(value, coder)); + public void broadcast(JavaSparkContext jsc) { + this.bcast = jsc.broadcast(value); + } } - 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; + /** + * A {@link com.cloudera.dataflow.spark.BroadcastHelper} that uses a + * {@link com.google.cloud.dataflow.sdk.coders.Coder} to encode values as byte arrays + * before broadcasting. + * @param + */ + static class CodedBroadcastHelper extends BroadcastHelper { + private Broadcast bcast; + private final Coder coder; + private transient T value; + + CodedBroadcastHelper(T value, Coder coder) { + this.value = value; + this.coder = coder; + } + + public synchronized T getValue() { + if (value == null) { + value = deserialize(); + } + return value; + } + + 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; } - return val; } } 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 index 195766e9dc02..e1af3cf75fb0 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -599,7 +599,7 @@ private static Map, BroadcastHelper> getSideInputs( for (PCollectionView view : views) { Iterable> collectionView = context.getPCollectionView(view); Coder>> coderInternal = view.getCoderInternal(); - BroadcastHelper helper = new BroadcastHelper<>(collectionView, coderInternal); + BroadcastHelper helper = BroadcastHelper.create(collectionView, coderInternal); //broadcast side inputs helper.broadcast(context.getSparkContext()); sideInputs.put(view.getTagInternal(), helper); From 79b08ad08ab8b57fa1b3309537d65d859865d264 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 9 Jul 2015 12:34:22 +0100 Subject: [PATCH 101/137] Make access of boolean system property clearer. (From Sean Owen.) --- .../main/java/com/cloudera/dataflow/spark/BroadcastHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 2622ce934d8d..27d23eb79b99 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -38,7 +38,7 @@ abstract class BroadcastHelper implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); public static BroadcastHelper create(T value, Coder coder) { - if (Boolean.getBoolean(DIRECT_BROADCAST)) { + if (Boolean.parseBoolean(System.getProperty(DIRECT_BROADCAST, "false"))) { return new DirectBroadcastHelper<>(value); } return new CodedBroadcastHelper<>(value, coder); From 3cae69bea6060c3ccf274a0444a93df3f8bc61b8 Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 10 Jul 2015 16:41:30 +0100 Subject: [PATCH 102/137] Only accumulate outputs from one call to processContext, rather than for the whole partition. Fixes https://github.com/cloudera/spark-dataflow/issues/61. --- .../cloudera/dataflow/spark/DoFnFunction.java | 22 +++---- .../dataflow/spark/MultiDoFnFunction.java | 36 +++++------ .../dataflow/spark/SparkProcessContext.java | 60 ++++++++++++++++++- 3 files changed, 90 insertions(+), 28 deletions(-) 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 index c5d7ddc77f5c..ae3dd795c933 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -51,20 +51,12 @@ class DoFnFunction implements FlatMapFunction, O> { @Override public Iterable call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); - //setup mFunction.startBundle(ctxt); ctxt.setup(); - //operation - while (iter.hasNext()) { - ctxt.element = iter.next(); - mFunction.processElement(ctxt); - } - //cleanup - mFunction.finishBundle(ctxt); - return ctxt.outputs; + return ctxt.getOutputIterable(iter, mFunction); } - private class ProcCtxt extends SparkProcessContext { + private class ProcCtxt extends SparkProcessContext { private final List outputs = new LinkedList<>(); @@ -77,5 +69,15 @@ private class ProcCtxt extends SparkProcessContext { public synchronized void output(O o) { outputs.add(o); } + + @Override + protected void clearOutput() { + outputs.clear(); + } + + protected Iterator getOutputIterator() { + return outputs.iterator(); + } } + } 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 index e6cb664e3b8c..8a9f8d53d7a9 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -21,7 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; -import com.google.common.collect.Iterables; +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; @@ -55,26 +55,12 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT @Override public Iterable, Object>> call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); - //setup mFunction.startBundle(ctxt); ctxt.setup(); - //operation - while (iter.hasNext()) { - ctxt.element = iter.next(); - mFunction.processElement(ctxt); - } - //cleanup - mFunction.finishBundle(ctxt); - return Iterables.transform(ctxt.outputs.entries(), - new Function, Object>, Tuple2, Object>>() { - @Override - public Tuple2, Object> apply(Map.Entry, Object> input) { - return new Tuple2, Object>(input.getKey(), input.getValue()); - } - }); + return ctxt.getOutputIterable(iter, mFunction); } - private class ProcCtxt extends SparkProcessContext { + private class ProcCtxt extends SparkProcessContext, Object>> { private final Multimap, Object> outputs = LinkedListMultimap.create(); @@ -97,5 +83,21 @@ public synchronized void sideOutput(TupleTag tag, T t) { public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { outputs.put(tupleTag, t); } + + @Override + protected void clearOutput() { + outputs.clear(); + } + + protected Iterator, Object>> getOutputIterator() { + return Iterators.transform(outputs.entries().iterator(), + new Function, Object>, Tuple2, Object>>() { + @Override + public Tuple2, Object> apply(Map.Entry, Object> input) { + return new Tuple2, Object>(input.getKey(), input.getValue()); + } + }); + } + } } 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 index 12fb4e043897..ee51c3507c70 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -34,11 +35,12 @@ import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.AbstractIterator; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -abstract class SparkProcessContext extends DoFn.ProcessContext { +abstract class SparkProcessContext extends DoFn.ProcessContext { private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class); @@ -192,4 +194,60 @@ public void writePCollectionViewData(TupleTag tag, }; } + protected abstract void clearOutput(); + protected abstract Iterator getOutputIterator(); + + protected Iterable getOutputIterable(final Iterator iter, final DoFn doFn) { + return new Iterable() { + @Override + public Iterator iterator() { + return new ProcCtxtIterator(iter, doFn); + } + }; + } + + private class ProcCtxtIterator extends AbstractIterator { + + private final Iterator inputIterator; + private final DoFn doFn; + private Iterator outputIterator; + + public ProcCtxtIterator(Iterator iterator, DoFn doFn) { + this.inputIterator = iterator; + this.doFn = doFn; + } + + @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 != null && outputIterator.hasNext()) { + return outputIterator.next(); + } + if (inputIterator.hasNext()) { + clearOutput(); + element = inputIterator.next(); + try { + doFn.processElement(SparkProcessContext.this); + } catch (Exception e) { + throw new IllegalStateException(e); + } + outputIterator = getOutputIterator(); + continue; // try to consume outputIterator from start of loop + } else { + try { + doFn.finishBundle(SparkProcessContext.this); + } catch (Exception e) { + throw new IllegalStateException(e); + } + return endOfData(); + } + } + } + } + } From c01421ce3375712bae0c0b88b5f2141c67284ade Mon Sep 17 00:00:00 2001 From: Tom White Date: Mon, 13 Jul 2015 21:08:57 +0100 Subject: [PATCH 103/137] Update to dataflow 0.4.150710. --- runners/spark/pom.xml | 2 +- .../com/cloudera/dataflow/spark/SparkProcessContext.java | 8 ++++---- .../com/cloudera/dataflow/spark/TransformTranslator.java | 8 ++++---- .../java/com/cloudera/dataflow/spark/NumShardsTest.java | 4 +++- .../com/cloudera/dataflow/spark/SerializationTest.java | 4 ++-- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 03821082aa10..3bce8c08d245 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,7 +23,7 @@ License. UTF-8 1.7 1.3.1 - 0.4.150602 + 0.4.150710 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 index ee51c3507c70..d0e9d6a6a01f 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -148,14 +148,14 @@ public KeyedState keyedState() { } @Override - public void store(CodedTupleTag tag, T value, Instant timestamp) - throws IOException { + public void writeToTagList(CodedTupleTag tag, T value) throws IOException { throw new UnsupportedOperationException( - "WindowingInternals#store() is not yet supported."); + "WindowingInternals#writeToTagList() is not yet supported."); } @Override - public void writeToTagList(CodedTupleTag tag, T value) throws IOException { + public void writeToTagList(CodedTupleTag tag, T value, Instant timestamp) + throws IOException { throw new UnsupportedOperationException( "WindowingInternals#writeToTagList() is not yet supported."); } 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 index e1af3cf75fb0..f1372181df73 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -520,10 +520,10 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { }; } - private static TransformEvaluator> create() { - return new TransformEvaluator>() { + private static TransformEvaluator> create() { + return new TransformEvaluator>() { @Override - public void evaluate(Create transform, EvaluationContext context) { + public void evaluate(Create.Values transform, EvaluationContext context) { Iterable elems = transform.getElements(); // Use a coder to convert the objects in the PCollection to byte arrays, so they // can be transferred over the network. @@ -624,7 +624,7 @@ private static Map, BroadcastHelper> getSideInputs( EVALUATORS.put(Combine.Globally.class, combineGlobally()); EVALUATORS.put(Combine.PerKey.class, combinePerKey()); EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl()); - EVALUATORS.put(Create.class, create()); + EVALUATORS.put(Create.Values.class, create()); EVALUATORS.put(View.AsSingleton.class, viewAsSingleton()); EVALUATORS.put(View.AsIterable.class, viewAsIter()); EVALUATORS.put(View.CreatePCollectionView.class, createPCollView()); 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 index 8985e663f2a4..9572b0f65392 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java @@ -20,6 +20,7 @@ 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.ParDo; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Charsets; import com.google.common.collect.Sets; @@ -62,7 +63,8 @@ public void testText() throws Exception { options.setRunner(SparkPipelineRunner.class); Pipeline p = Pipeline.create(options); PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); - PCollection output = inputWords.apply(new WordCount.CountWords()); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(ParDo.of(new WordCount.FormatAsTextFn())); output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt")); EvaluationResult res = SparkPipelineRunner.create().run(p); res.close(); 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 index 40591e59aa1a..bd1a4e8cfdb7 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -112,8 +112,8 @@ public void testRun() throws Exception { SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); options.setRunner(SparkPipelineRunner.class); Pipeline p = Pipeline.create(options); - PCollection inputWords = p.apply(Create.of(WORDS)).setCoder - (StringHolderUtf8Coder.of()); + PCollection inputWords = + p.apply(Create.of(WORDS).withCoder(StringHolderUtf8Coder.of())); PCollection output = inputWords.apply(new CountWords()); DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); From 4415862278948d44628535db4d1071e863783deb Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 14 Jul 2015 10:48:40 +0100 Subject: [PATCH 104/137] Prevent possible NPE. --- .../java/com/cloudera/dataflow/spark/TransformTranslator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 index f1372181df73..2c61a42f2e89 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -44,7 +44,6 @@ 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.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.avro.mapred.AvroKey; @@ -206,7 +205,8 @@ public void evaluate(Combine.Globally transform, EvaluationContext context Coder coder = context.getOutput(transform).getCoder(); JavaRDD outRdd = context.getSparkContext().parallelize( - CoderHelpers.toByteArrays(ImmutableList.of(output), coder)); + // 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))); } }; From ebf70534cf7f9dd324220cb437f14072c2e93d39 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 14 Jul 2015 14:08:28 +0100 Subject: [PATCH 105/137] [maven-release-plugin] prepare release spark-dataflow-0.2.2 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 3bce8c08d245..e538d8262b89 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.2-SNAPSHOT + 0.2.2 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.2.2 From 72167a2c5152df174eb25d46eb3caea6f408ea56 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 14 Jul 2015 14:08:34 +0100 Subject: [PATCH 106/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index e538d8262b89..ec833ef227f4 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.2 + 0.2.3-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.2.2 + HEAD From 27349adc2d353a90e9ec3dcc1ccf2b941bf09fe1 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 14 Jul 2015 14:14:47 +0100 Subject: [PATCH 107/137] Update README to latest version (0.2.2). --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index d93f55470383..0bf1ec817c3a 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.2.0 + 0.2.2 and are hosted in Cloudera's repository at: From 76815589f5d4b96868b8438f1820d17e0a5822ab Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 14 Jul 2015 16:44:15 +0100 Subject: [PATCH 108/137] Fix bug where values written to the output in DoFn#startBundle and DoFn#finishBundle were being ignored. Introduced in 62830a0. --- .../cloudera/dataflow/spark/DoFnFunction.java | 2 +- .../dataflow/spark/SparkProcessContext.java | 22 ++++--- .../dataflow/spark/DoFnOutputTest.java | 57 +++++++++++++++++++ 3 files changed, 73 insertions(+), 8 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.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 index ae3dd795c933..542f2ecc24b7 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -51,8 +51,8 @@ class DoFnFunction implements FlatMapFunction, O> { @Override public Iterable call(Iterator iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); - mFunction.startBundle(ctxt); ctxt.setup(); + mFunction.startBundle(ctxt); return ctxt.getOutputIterable(iter, mFunction); } 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 index d0e9d6a6a01f..bda838c3e6c9 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -211,10 +211,12 @@ private class ProcCtxtIterator extends AbstractIterator { private final Iterator inputIterator; private final DoFn doFn; private Iterator outputIterator; + private boolean calledFinish = false; public ProcCtxtIterator(Iterator iterator, DoFn doFn) { this.inputIterator = iterator; this.doFn = doFn; + this.outputIterator = getOutputIterator(); } @Override @@ -225,10 +227,9 @@ protected V computeNext() { // 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 != null && outputIterator.hasNext()) { + if (outputIterator.hasNext()) { return outputIterator.next(); - } - if (inputIterator.hasNext()) { + } else if (inputIterator.hasNext()) { clearOutput(); element = inputIterator.next(); try { @@ -239,10 +240,17 @@ protected V computeNext() { outputIterator = getOutputIterator(); continue; // try to consume outputIterator from start of loop } else { - try { - doFn.finishBundle(SparkProcessContext.this); - } catch (Exception e) { - throw new IllegalStateException(e); + // 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 IllegalStateException(e); + } + outputIterator = getOutputIterator(); + continue; // try to consume outputIterator from start of loop } return endOfData(); } 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 new file mode 100644 index 000000000000..2b0947f115ed --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/DoFnOutputTest.java @@ -0,0 +1,57 @@ +/* + * 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 strings = pipeline.apply(Create.of("a")); + // Test that values written from startBundle() and finishBundle() are written to + // the output + PCollection output = strings.apply(ParDo.of(new DoFn() { + @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(); + } +} From fe0b8e9a9bfa3c01090d8d81be6d0a766bc6d148 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 16 Jul 2015 11:35:22 +0100 Subject: [PATCH 109/137] [maven-release-plugin] prepare release spark-dataflow-0.2.3 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index ec833ef227f4..17f9d1a52feb 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.3-SNAPSHOT + 0.2.3 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.2.3 From 4ec8c606704f754240d0e1b6ad2261409038e13e Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 16 Jul 2015 11:35:28 +0100 Subject: [PATCH 110/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 17f9d1a52feb..61a9226b4752 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.3 + 0.2.4-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.2.3 + HEAD From 3b1441f4a1c71998bd158b7b82f6a1f9b362f8e5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 21 Jul 2015 13:06:36 +0100 Subject: [PATCH 111/137] Avoid warning email by not running codecov unless it was configured; update jacoco and shade plugins --- runners/spark/.travis.yml | 2 +- runners/spark/pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/spark/.travis.yml b/runners/spark/.travis.yml index 6b91c9e4dacb..2e6e3e228560 100644 --- a/runners/spark/.travis.yml +++ b/runners/spark/.travis.yml @@ -13,4 +13,4 @@ matrix: cache: directories: - $HOME/.m2 -after_success: bash <(curl -s https://codecov.io/bash) +after_success: if [ -n "$JACOCO" ]; then bash <(curl -s https://codecov.io/bash); fi diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 61a9226b4752..ee660516f8c9 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -205,7 +205,7 @@ License. org.jacoco jacoco-maven-plugin - 0.7.4.201502262128 + 0.7.5.201505241946 @@ -224,7 +224,7 @@ License. org.apache.maven.plugins maven-shade-plugin - 2.3 + 2.4.1 package From d7a35bdf260e8a13ddb8b0c4ab5ee7a3ed3015df Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 28 Jul 2015 16:14:31 -0700 Subject: [PATCH 112/137] [maven-release-plugin] prepare release spark-dataflow-0.3.0 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index ee660516f8c9..1f06092a2c34 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.2.4-SNAPSHOT + 0.3.0 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.3.0 From 383eeebe5ec2080f419dd8533b09d8dd9ab21ca5 Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 28 Jul 2015 16:14:48 -0700 Subject: [PATCH 113/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 1f06092a2c34..de9efb9cedef 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.3.0 + 0.3.1-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.3.0 + HEAD From b83d6660bac73c22f1b0d27cd830f38db34d7f2e Mon Sep 17 00:00:00 2001 From: Tom White Date: Tue, 28 Jul 2015 16:19:15 -0700 Subject: [PATCH 114/137] Update README to latest version (0.3.0). --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 0bf1ec817c3a..a8ff5a140483 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.2.2 + 0.3.0 and are hosted in Cloudera's repository at: From 89945bf676affe2cd52fed91551cb1037fc2faae Mon Sep 17 00:00:00 2001 From: Tom White Date: Wed, 5 Aug 2015 18:10:59 +0100 Subject: [PATCH 115/137] Update to dataflow 0.4.150727. --- runners/spark/pom.xml | 2 +- .../dataflow/spark/SparkProcessContext.java | 53 ++++++------------- .../dataflow/spark/TransformTranslator.java | 8 +-- .../cloudera/dataflow/spark/TfIdfTest.java | 2 +- .../spark/TransformTranslatorTest.java | 5 ++ 5 files changed, 24 insertions(+), 46 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index de9efb9cedef..74b0fed83eda 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,7 +23,7 @@ License. UTF-8 1.7 1.3.1 - 0.4.150710 + 0.4.150727 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 index bda838c3e6c9..259f90ccc915 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; -import java.util.List; import java.util.Map; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -29,10 +28,11 @@ 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.GlobalWindow; -import com.google.cloud.dataflow.sdk.util.TimerManager; +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.values.CodedTupleTag; +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; @@ -125,6 +125,11 @@ public BoundedWindow window() { return GlobalWindow.INSTANCE; } + @Override + public PaneInfo pane() { + return PaneInfo.DEFAULT; + } + @Override public WindowingInternals windowingInternals() { return new WindowingInternals() { @@ -136,53 +141,25 @@ public Collection windows() { @Override public void outputWindowedValue(O output, Instant timestamp, Collection windows) { + extends BoundedWindow> windows, PaneInfo paneInfo) { output(output); } @Override - public KeyedState keyedState() { + public StateInternals stateInternals() { throw new UnsupportedOperationException( - "WindowingInternals#keyedState() is not yet supported."); - + "WindowingInternals#stateInternals() is not yet supported."); } @Override - public void writeToTagList(CodedTupleTag tag, T value) throws IOException { + public TimerInternals timerInternals() { throw new UnsupportedOperationException( - "WindowingInternals#writeToTagList() is not yet supported."); + "WindowingInternals#timerInternals() is not yet supported."); } @Override - public void writeToTagList(CodedTupleTag tag, T value, Instant timestamp) - throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#writeToTagList() is not yet supported."); - } - - @Override - public void deleteTagList(CodedTupleTag tag) { - throw new UnsupportedOperationException( - "WindowingInternals#deleteTagList() is not yet supported."); - } - - @Override - public Iterable readTagList(CodedTupleTag tag) throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#readTagList() is not yet supported."); - } - - @Override - public Map, Iterable> readTagList(List> tags) - throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#readTagList() is not yet supported."); - } - - @Override - public TimerManager getTimerManager() { - throw new UnsupportedOperationException( - "WindowingInternals#getTimerManager() is not yet supported."); + public PaneInfo pane() { + return PaneInfo.DEFAULT; } @Override 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 index 2c61a42f2e89..ee300fd5021d 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -35,7 +35,6 @@ 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.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.KV; @@ -511,11 +510,8 @@ private static TransformEvaluator> window() { return new TransformEvaluator>() { @Override public void evaluate(Window.Bound transform, EvaluationContext context) { - if (transform.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) { - context.setOutputRDD(transform, context.getInputRDD(transform)); - } else { - throw new UnsupportedOperationException("Non-global windowing not supported"); - } + // TODO: detect and support non-global windows + context.setOutputRDD(transform, context.getInputRDD(transform)); } }; } 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 index 680d8b70492a..35ab26e598af 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TfIdfTest.java @@ -15,7 +15,7 @@ package com.cloudera.dataflow.spark; -import com.google.cloud.dataflow.examples.TfIdf; +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; 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 index 0251808b441b..540bdd9d1a3b 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java @@ -23,6 +23,7 @@ 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; @@ -76,6 +77,10 @@ public void testTextIOReadAndWriteTransforms() throws IOException { List 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()); } From 5ec8d59c68eed3c27e03b278327b1696ff6c5256 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 6 Aug 2015 07:20:34 +0100 Subject: [PATCH 116/137] [maven-release-plugin] prepare release spark-dataflow-0.4.0 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 74b0fed83eda..655c2876ef8d 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.3.1-SNAPSHOT + 0.4.0 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.4.0 From 1fdf602b0a04e261ffbf528e2ed9e17f0c4fdf92 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 6 Aug 2015 07:20:41 +0100 Subject: [PATCH 117/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 655c2876ef8d..d055feaeb1ee 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.4.0 + 0.4.1-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.4.0 + HEAD From 922508c090139c8dcd772625c066f7a010114efa Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 6 Aug 2015 07:53:10 +0100 Subject: [PATCH 118/137] Update README to latest version (0.4.0). --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index a8ff5a140483..0de52e172f75 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.3.0 + 0.4.0 and are hosted in Cloudera's repository at: From 27fd2904240c58fcb492477e86fed307089d4cac Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 13 Aug 2015 16:58:32 +0100 Subject: [PATCH 119/137] Dataflow goes GA! Update to version 1.0.0. --- runners/spark/pom.xml | 2 +- .../java/com/cloudera/dataflow/spark/SparkProcessContext.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index d055feaeb1ee..fadb372ec678 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,7 +23,7 @@ License. UTF-8 1.7 1.3.1 - 0.4.150727 + 1.0.0 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 index 259f90ccc915..7777f21eb869 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -127,7 +127,7 @@ public BoundedWindow window() { @Override public PaneInfo pane() { - return PaneInfo.DEFAULT; + return PaneInfo.NO_FIRING; } @Override @@ -159,7 +159,7 @@ public TimerInternals timerInternals() { @Override public PaneInfo pane() { - return PaneInfo.DEFAULT; + return PaneInfo.NO_FIRING; } @Override From 3e767f5a8339960ea3fdbe1fefb05ae8b742c5d7 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 13 Aug 2015 17:03:05 +0100 Subject: [PATCH 120/137] [maven-release-plugin] prepare release spark-dataflow-0.4.1 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index fadb372ec678..b66775c17196 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.4.1-SNAPSHOT + 0.4.1 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.4.1 From 4536853d070ad14b8e570ee011c28619df4ee55e Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 13 Aug 2015 17:03:11 +0100 Subject: [PATCH 121/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index b66775c17196..0eeeea9a3f0c 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.4.1 + 0.4.2-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.4.1 + HEAD From 4b98c163b962a5f9d8744e4fcd8a7301855731fd Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 13 Aug 2015 20:51:53 +0100 Subject: [PATCH 122/137] Update README to latest version (0.4.1). --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 0de52e172f75..973912199d01 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.4.0 + 0.4.1 and are hosted in Cloudera's repository at: From 0c84c9d7023051d47e936c45058d2b6146a5c9b3 Mon Sep 17 00:00:00 2001 From: Amit Sela Date: Mon, 17 Aug 2015 21:40:32 +0300 Subject: [PATCH 123/137] Correct input parameter is --inputFile --- runners/spark/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 0eeeea9a3f0c..d1bd4ffbde54 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -195,7 +195,7 @@ License. -classpath ${mainClass} - --input=${input} + --inputFile=${input} --output=${output} --runner=${runner} --sparkMaster=${sparkMaster} From 783886592a3f6baf409fb06a75fe4c4994def74a Mon Sep 17 00:00:00 2001 From: Tom White Date: Fri, 5 Jun 2015 09:14:16 +0200 Subject: [PATCH 124/137] Add support for writes with HadoopIO. This allows Hadoop FileOutputFormats to be used with Spark Dataflow, as long as they implement the ShardNameTemplateAware interface. This is easily achieved by subclassing the desired FileOutputFormat class, see TemplatedSequenceFileOutputFormat for an example. --- runners/spark/build-resources/checkstyle.xml | 6 +- .../cloudera/dataflow/hadoop/HadoopIO.java | 114 +++++++++++++++ .../dataflow/hadoop/WritableCoder.java | 112 ++++++++++++++ .../spark/ShardNameTemplateAware.java | 28 ++++ .../spark/ShardNameTemplateHelper.java | 58 ++++++++ .../spark/TemplatedAvroKeyOutputFormat.java | 40 +++++ .../TemplatedSequenceFileOutputFormat.java | 40 +++++ .../spark/TemplatedTextOutputFormat.java | 24 +-- .../dataflow/spark/TransformTranslator.java | 137 +++++++++++++----- .../dataflow/spark/AvroPipelineTest.java | 4 +- .../spark/HadoopFileFormatPipelineTest.java | 46 +++--- 11 files changed, 530 insertions(+), 79 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateAware.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/ShardNameTemplateHelper.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedAvroKeyOutputFormat.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedSequenceFileOutputFormat.java diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml index 716270a6f377..c5b884d7a9dc 100644 --- a/runners/spark/build-resources/checkstyle.xml +++ b/runners/spark/build-resources/checkstyle.xml @@ -115,7 +115,9 @@ - + + + @@ -180,7 +182,7 @@ - + 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 index 533dd3065894..6389db368fad 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -14,13 +14,21 @@ */ 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 { @@ -85,4 +93,110 @@ public PCollection> apply(PInput input) { } } + + public static final class Write { + + private Write() { + } + + public static Bound to(String filenamePrefix, + Class> format, Class key, Class value) { + return new Bound<>(filenamePrefix, format, key, value); + } + + public static class Bound extends PTransform>, 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> formatClass; + private final Class keyClass; + private final Class valueClass; + private final Map configurationProperties; + + Bound(String filenamePrefix, Class> format, + Class key, + Class value) { + this(filenamePrefix, "", 0, ShardNameTemplate.INDEX_OF_MAX, format, key, value, + new HashMap()); + } + + Bound(String filenamePrefix, String filenameSuffix, int numShards, + String shardTemplate, Class> format, + Class key, Class value, Map 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 withoutSharding() { + return new Bound<>(filenamePrefix, filenameSuffix, 1, "", formatClass, + keyClass, valueClass, configurationProperties); + } + + public Bound 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> getFormatClass() { + return formatClass; + } + + public Class getValueClass() { + return valueClass; + } + + public Class getKeyClass() { + return keyClass; + } + + public Map getConfigurationProperties() { + return configurationProperties; + } + + @Override + public PDone apply(PCollection> 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()); + } + } + } } 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 new file mode 100644 index 000000000000..dbc6779353f6 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java @@ -0,0 +1,112 @@ +/* + * 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.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.Writable; + +/** + * A {@code WritableCoder} is a {@link com.google.cloud.dataflow.sdk.coders.Coder} for a + * Java class that implements {@link org.apache.hadoop.io.Writable}. + * + *

To use, specify the coder type on a PCollection: + *

+ * {@code
+ *   PCollection records =
+ *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
+ * }
+ * 
+ * + * @param the type of elements handled by this coder + */ +public class WritableCoder extends StandardCoder { + private static final long serialVersionUID = 0L; + + /** + * Returns a {@code WritableCoder} instance for the provided element class. + * @param the element type + * @param clazz the element class + * @return a {@code WritableCoder} instance for the provided element class + */ + public static WritableCoder of(Class clazz) { + 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) clazz); + } + + private final Class type; + + public WritableCoder(Class 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.newInstance(); + t.readFields(new DataInputStream(inStream)); + return t; + } catch (InstantiationException | IllegalAccessException e) { + throw new CoderException("unable to deserialize record", e); + } + } + + @Override + public List> 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."); + } + +} 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 new file mode 100644 index 000000000000..bb9a7a558e85 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/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 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 { +} 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 new file mode 100644 index 000000000000..56980a1e0e56 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/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 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 Path getDefaultWorkFile(FileOutputFormat 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; + } + +} 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 new file mode 100644 index 000000000000..ef24137b9f62 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/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 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 extends AvroKeyOutputFormat + 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); + } + +} 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 new file mode 100644 index 000000000000..3ab07b5a95b1 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/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 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 extends SequenceFileOutputFormat + 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); + } + +} 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 index 5d00900aeac4..a8e218d40bf3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TemplatedTextOutputFormat.java @@ -20,17 +20,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; 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.TextOutputFormat; -import static com.cloudera.dataflow.spark.ShardNameBuilder.replaceShardNumber; - -public class TemplatedTextOutputFormat extends TextOutputFormat { - - public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.textoutputformat.prefix"; - public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.textoutputformat.template"; - public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.textoutputformat.suffix"; +public class TemplatedTextOutputFormat extends TextOutputFormat + implements ShardNameTemplateAware { @Override public void checkOutputSpecs(JobContext job) { @@ -41,18 +34,7 @@ public void checkOutputSpecs(JobContext job) { public Path getDefaultWorkFile(TaskAttemptContext context, String extension) throws IOException { // note that the passed-in extension is ignored since it comes from the template - FileOutputCommitter committer = - (FileOutputCommitter) getOutputCommitter(context); - return new Path(committer.getWorkPath(), getOutputFile(context)); + return ShardNameTemplateHelper.getDefaultWorkFile(this, context); } - private 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; - } } 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 index ee300fd5021d..dfb01f162931 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -48,11 +48,11 @@ import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; -import org.apache.avro.mapreduce.AvroKeyOutputFormat; 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; @@ -409,27 +409,12 @@ public Tuple2 call(T t) throws Exception { return new Tuple2<>(t, null); } }); - int shardCount = transform.getNumShards(); - if (shardCount == 0) { - // use default number of shards, but find the actual number for the template - shardCount = last.partitions().size(); - } else { - // number of shards was set explicitly, so repartition - last = last.repartition(transform.getNumShards()); - } - - String template = replaceShardCount(transform.getShardTemplate(), shardCount); - String outputDir = getOutputDirectory(transform.getFilenamePrefix(), template); - String filePrefix = getOutputFilePrefix(transform.getFilenamePrefix(), template); - String fileTemplate = getOutputFileTemplate(transform.getFilenamePrefix(), template); - String fileSuffix = transform.getFilenameSuffix(); - - Configuration conf = new Configuration(); - conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_PREFIX, filePrefix); - conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_TEMPLATE, fileTemplate); - conf.set(TemplatedTextOutputFormat.OUTPUT_FILE_SUFFIX, fileSuffix); - last.saveAsNewAPIHadoopFile(outputDir, Text.class, NullWritable.class, - TemplatedTextOutputFormat.class, conf); + ShardTemplateInformation shardTemplateInfo = + new ShardTemplateInformation(transform.getNumShards(), + transform.getShardTemplate(), transform.getFilenamePrefix(), + transform.getFilenameSuffix()); + writeHadoopFile(last, new Configuration(), shardTemplateInfo, Text.class, + NullWritable.class, TemplatedTextOutputFormat.class); } }; } @@ -462,9 +447,6 @@ private static TransformEvaluator> writeAvro() { return new TransformEvaluator>() { @Override public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) { - String pattern = transform.getFilenamePrefix(); - @SuppressWarnings("unchecked") - JavaRDDLike last = (JavaRDDLike) context.getInputRDD(transform); Job job; try { job = Job.getInstance(); @@ -472,14 +454,21 @@ public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) throw new IllegalStateException(e); } AvroJob.setOutputKeySchema(job, transform.getSchema()); - last.mapToPair(new PairFunction, NullWritable>() { - @Override - public Tuple2, NullWritable> call(T t) throws Exception { - return new Tuple2<>(new AvroKey<>(t), NullWritable.get()); - }}) - .saveAsNewAPIHadoopFile(pattern, AvroKey.class, NullWritable.class, - AvroKeyOutputFormat.class, job.getConfiguration()); - + @SuppressWarnings("unchecked") + JavaPairRDD, NullWritable> last = + ((JavaRDDLike) context.getInputRDD(transform)) + .mapToPair(new PairFunction, NullWritable>() { + @Override + public Tuple2, 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); } }; } @@ -506,6 +495,87 @@ public KV call(Tuple2 t2) throws Exception { }; } + private static TransformEvaluator> writeHadoop() { + return new TransformEvaluator>() { + @Override + public void evaluate(HadoopIO.Write.Bound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaPairRDD last = ((JavaRDDLike, ?>) context + .getInputRDD(transform)) + .mapToPair(new PairFunction, K, V>() { + @Override + public Tuple2 call(KV 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 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; + } + + public int getNumShards() { + return numShards; + } + + public String getShardTemplate() { + return shardTemplate; + } + + public String getFilenamePrefix() { + return filenamePrefix; + } + + public String getFilenameSuffix() { + return filenameSuffix; + } + } + + private static void writeHadoopFile(JavaPairRDD rdd, Configuration conf, + ShardTemplateInformation shardTemplateInfo, Class keyClass, Class valueClass, + Class 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 TransformEvaluator> window() { return new TransformEvaluator>() { @Override @@ -613,6 +683,7 @@ private static Map, BroadcastHelper> getSideInputs( 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()); 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 index 95f100c1c390..ea4cc38e39ee 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/AvroPipelineTest.java @@ -90,8 +90,8 @@ private void populateGenericFile(List genericRecords, Schema sche private List readGenericFile() throws IOException { List records = Lists.newArrayList(); GenericDatumReader genericDatumReader = new GenericDatumReader<>(); - try (DataFileReader dataFileReader = new DataFileReader<> - (new File(outputDir, "part-r-00000.avro"), genericDatumReader)) { + try (DataFileReader dataFileReader = + new DataFileReader<>(new File(outputDir + "-00000-of-00001"), genericDatumReader)) { for (GenericRecord record : dataFileReader) { records.add(record); } 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 index ba6f7b0d1be5..b35101873187 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java @@ -17,26 +17,22 @@ import com.cloudera.dataflow.hadoop.HadoopIO; 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.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.common.base.Charsets; -import com.google.common.io.Files; import java.io.File; import java.io.IOException; -import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; 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; @@ -60,24 +56,39 @@ public void setUp() throws IOException { } @Test - public void testGeneric() throws Exception { + public void testSequenceFile() throws Exception { populateFile(); Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); @SuppressWarnings("unchecked") Class> inputFormatClass = (Class>) (Class) SequenceFileInputFormat.class; - HadoopIO.Read.Bound bound = + HadoopIO.Read.Bound read = HadoopIO.Read.from(inputFile.getAbsolutePath(), inputFormatClass, IntWritable.class, Text.class); - PCollection> input = p.apply(bound); - input.apply(ParDo.of(new TabSeparatedString())) - .apply(TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding()); + PCollection> input = p.apply(read); + @SuppressWarnings("unchecked") + Class> outputFormatClass = + (Class>) (Class) TemplatedSequenceFileOutputFormat.class; + @SuppressWarnings("unchecked") + HadoopIO.Write.Bound write = HadoopIO.Write.to(outputFile.getAbsolutePath(), + outputFormatClass, IntWritable.class, Text.class); + input.apply(write.withoutSharding()); EvaluationResult res = SparkPipelineRunner.create().run(p); res.close(); - List records = Files.readLines(outputFile, Charsets.UTF_8); - for (int i = 0; i < 5; i++) { - assertEquals(i + "\tvalue-" + i, records.get(i)); + IntWritable key = new IntWritable(); + Text value = new Text(); + Reader reader = null; + try { + 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++; + } + } finally { + IOUtils.closeStream(reader); } } @@ -99,11 +110,4 @@ private void populateFile() throws IOException { } } - static class TabSeparatedString extends DoFn, String> { - @Override - public void processElement(ProcessContext c) throws Exception { - c.output(c.element().getKey().toString() + "\t" + c.element().getValue().toString()); - } - } - } From 8762b26717e981d02a44e2df5ddd3ef08f10e2f9 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 20 Aug 2015 12:54:38 +0100 Subject: [PATCH 125/137] Add NullWritableCoder and test. --- .../dataflow/hadoop/NullWritableCoder.java | 71 +++++++++++++++++++ .../dataflow/hadoop/WritableCoder.java | 4 ++ .../dataflow/hadoop/WritableCoderTest.java | 42 +++++++++++ 3 files changed, 117 insertions(+) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/hadoop/NullWritableCoder.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/hadoop/WritableCoderTest.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 new file mode 100644 index 000000000000..5e5d3919ac53 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/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 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 { + 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 + } +} 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 index dbc6779353f6..ea47109dcdb4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java @@ -28,6 +28,7 @@ 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; /** @@ -54,6 +55,9 @@ public class WritableCoder extends StandardCoder { * @return a {@code WritableCoder} instance for the provided element class */ public static WritableCoder of(Class clazz) { + if (clazz.equals(NullWritable.class)) { + return (WritableCoder) NullWritableCoder.of(); + } return new WritableCoder<>(clazz); } 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 new file mode 100644 index 000000000000..29a73b65b4f0 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/hadoop/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 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 coder = WritableCoder.of(IntWritable.class); + + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + + @Test + public void testNullWritableEncoding() throws Exception { + WritableCoder coder = WritableCoder.of(NullWritable.class); + + CoderProperties.coderDecodeEncodeEqual(coder, NullWritable.get()); + } +} From b8949b810bb388a2e71549e3a5e03672ce2cdf2b Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 20 Aug 2015 15:33:23 +0100 Subject: [PATCH 126/137] [maven-release-plugin] prepare release spark-dataflow-0.4.2 --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index d1bd4ffbde54..baa5d70709bd 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.4.2-SNAPSHOT + 0.4.2 jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - HEAD + spark-dataflow-0.4.2 From ecc33d8310101e629137ad42cd8833cb9d3cb935 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 20 Aug 2015 15:33:29 +0100 Subject: [PATCH 127/137] [maven-release-plugin] prepare for next development iteration --- runners/spark/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index baa5d70709bd..df21c437dabc 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -15,7 +15,7 @@ License. Dataflow on Spark com.cloudera.dataflow.spark spark-dataflow - 0.4.2 + 0.4.3-SNAPSHOT jar @@ -385,7 +385,7 @@ License. scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git scm:git:https://github.com/cloudera/spark-dataflow.git - spark-dataflow-0.4.2 + HEAD From 90c49b4fc034729fffd02ac9d3a291e3ac86aa61 Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 20 Aug 2015 15:45:03 +0100 Subject: [PATCH 128/137] Update README to latest version (0.4.2). --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index 973912199d01..d3429fe1aaef 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -34,7 +34,7 @@ The Maven coordinates of the current version of this project are: com.cloudera.dataflow.spark spark-dataflow - 0.4.1 + 0.4.2 and are hosted in Cloudera's repository at: From 877970158ff07eb547427bc81b7ad1209577c74c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 8 Oct 2015 22:40:55 +0100 Subject: [PATCH 129/137] Add tests for Spark 1.4 / 1.5 in Travis --- runners/spark/.travis.yml | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/runners/spark/.travis.yml b/runners/spark/.travis.yml index 2e6e3e228560..c4af8a6769a1 100644 --- a/runners/spark/.travis.yml +++ b/runners/spark/.travis.yml @@ -4,9 +4,15 @@ install: mvn ${JAVA} ${SPARK} -DskipTests=true -Dmaven.javadoc.skip=true -B -V i script: mvn ${JAVA} ${SPARK} ${JACOCO} -Dmaven.javadoc.skip=true -B verify matrix: include: - # Covers Java 7, Open JDK, and code coverage + # 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 From 1c603d1c526e0610a3ad4edee9afe4f7ee4fd1e8 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 8 Oct 2015 22:35:25 +0100 Subject: [PATCH 130/137] Fix a few Coverity inspection results plus more IntelliJ results --- .../com/cloudera/dataflow/hadoop/WritableCoder.java | 11 ++++++++--- .../com/cloudera/dataflow/spark/BroadcastHelper.java | 4 ++++ .../java/com/cloudera/dataflow/spark/ByteArray.java | 2 +- .../cloudera/dataflow/spark/EvaluationContext.java | 4 ++-- .../cloudera/dataflow/spark/SparkProcessContext.java | 5 ++--- .../cloudera/dataflow/spark/TransformTranslator.java | 4 ++-- .../cloudera/dataflow/spark/CombineGloballyTest.java | 2 +- .../dataflow/spark/MultiOutputWordCountTest.java | 5 +++-- .../cloudera/dataflow/spark/SerializationTest.java | 4 ++-- .../com/cloudera/dataflow/spark/SideEffectsTest.java | 5 ++--- 10 files changed, 27 insertions(+), 19 deletions(-) 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 index ea47109dcdb4..759fb581ef6b 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java @@ -20,6 +20,7 @@ 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; @@ -56,7 +57,9 @@ public class WritableCoder extends StandardCoder { */ public static WritableCoder of(Class clazz) { if (clazz.equals(NullWritable.class)) { - return (WritableCoder) NullWritableCoder.of(); + @SuppressWarnings("unchecked") + WritableCoder result = (WritableCoder) NullWritableCoder.of(); + return result; } return new WritableCoder<>(clazz); } @@ -87,11 +90,13 @@ public void encode(T value, OutputStream outStream, Context context) throws IOEx @Override public T decode(InputStream inStream, Context context) throws IOException { try { - T t = type.newInstance(); + T t = type.getConstructor().newInstance(); t.readFields(new DataInputStream(inStream)); return t; - } catch (InstantiationException | IllegalAccessException e) { + } 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()); } } 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 index 27d23eb79b99..6ef70f328183 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java @@ -62,6 +62,7 @@ static class DirectBroadcastHelper extends BroadcastHelper { this.value = value; } + @Override public synchronized T getValue() { if (value == null) { value = bcast.getValue(); @@ -69,6 +70,7 @@ public synchronized T getValue() { return value; } + @Override public void broadcast(JavaSparkContext jsc) { this.bcast = jsc.broadcast(value); } @@ -90,6 +92,7 @@ static class CodedBroadcastHelper extends BroadcastHelper { this.coder = coder; } + @Override public synchronized T getValue() { if (value == null) { value = deserialize(); @@ -97,6 +100,7 @@ public synchronized T getValue() { return value; } + @Override public void broadcast(JavaSparkContext jsc) { this.bcast = jsc.broadcast(CoderHelpers.toByteArray(value, coder)); } 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 index 1db0a8bac5a4..06db57206d43 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/ByteArray.java @@ -23,7 +23,7 @@ class ByteArray implements Serializable, Comparable { private final byte[] value; - public ByteArray(byte[] value) { + ByteArray(byte[] value) { this.value = value; } 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 index 649cbe9088e3..eb9554fd14b3 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -78,12 +78,12 @@ private class RDDHolder { private Coder coder; private JavaRDDLike rdd; - public RDDHolder(Iterable values, Coder coder) { + RDDHolder(Iterable values, Coder coder) { this.values = values; this.coder = coder; } - public RDDHolder(JavaRDDLike rdd) { + RDDHolder(JavaRDDLike rdd) { this.rdd = rdd; } 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 index 7777f21eb869..ee2235aaecc4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -61,7 +61,7 @@ abstract class SparkProcessContext extends DoFn.ProcessContext { } void setup() { - super.setupDelegateAggregators(); + setupDelegateAggregators(); } @Override @@ -190,7 +190,7 @@ private class ProcCtxtIterator extends AbstractIterator { private Iterator outputIterator; private boolean calledFinish = false; - public ProcCtxtIterator(Iterator iterator, DoFn doFn) { + ProcCtxtIterator(Iterator iterator, DoFn doFn) { this.inputIterator = iterator; this.doFn = doFn; this.outputIterator = getOutputIterator(); @@ -215,7 +215,6 @@ protected V computeNext() { throw new IllegalStateException(e); } outputIterator = getOutputIterator(); - continue; // try to consume outputIterator from start of loop } else { // no more input to consume, but finishBundle can produce more output if (!calledFinish) { 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 index dfb01f162931..4537aa45284e 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -124,8 +124,8 @@ public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContex (JavaRDDLike, ?>) context.getInputRDD(transform); @SuppressWarnings("unchecked") KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); - final Coder keyCoder = coder.getKeyCoder(); - final Coder valueCoder = coder.getValueCoder(); + Coder keyCoder = coder.getKeyCoder(); + Coder 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. 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 index be5f6dc7c2d5..51ba905c4dac 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java @@ -66,7 +66,7 @@ public StringBuilder mergeAccumulators(Iterable accumulators) { StringBuilder sb = new StringBuilder(); for (StringBuilder accum : accumulators) { if (accum != null) { - sb.append(accum.toString()); + sb.append(accum); } } return sb; 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 index bf2ecdcad742..179816d73e96 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -65,6 +65,7 @@ public void testRun() throws Exception { EvaluationResult res = SparkPipelineRunner.create().run(p); Iterable> actualLower = res.get(luc.get(lowerCnts)); + Assert.assertEquals("and", actualLower.iterator().next().getKey()); Iterable> actualUpper = res.get(luc.get(upperCnts)); Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); Iterable actualUniqCount = res.get(unique); @@ -85,9 +86,9 @@ public void testRun() throws Exception { */ static class ExtractWordsFn extends DoFn { - private Aggregator totalWords = createAggregator("totalWords", + private final Aggregator totalWords = createAggregator("totalWords", new Sum.SumIntegerFn()); - private Aggregator maxWordLength = createAggregator("maxWordLength", + private final Aggregator maxWordLength = createAggregator("maxWordLength", new Max.MaxIntegerFn()); private final PCollectionView regex; 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 index bd1a4e8cfdb7..a8edb3a068b4 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java @@ -43,7 +43,7 @@ public class SerializationTest { public static class StringHolder { // not serializable - private String string; + private final String string; public StringHolder(String string) { this.string = string; @@ -71,7 +71,7 @@ public String toString() { public static class StringHolderUtf8Coder extends AtomicCoder { - private StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of(); + private final StringUtf8Coder stringUtf8Coder = StringUtf8Coder.of(); @Override public void encode(StringHolder value, OutputStream outStream, Context context) throws IOException { 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 index 7292bf083994..666737dc43ff 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java @@ -37,7 +37,7 @@ public void test() throws Exception { pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); PCollection strings = pipeline.apply(Create.of("a")); - PCollection output = strings.apply(ParDo.of(new DoFn() { + strings.apply(ParDo.of(new DoFn() { @Override public void processElement(ProcessContext c) throws Exception { throw new IllegalStateException("Side effect"); @@ -49,7 +49,6 @@ public void processElement(ProcessContext c) throws Exception { fail("Run should thrown an exception"); } catch (Exception e) { // expected - e.printStackTrace(); } } -} +} \ No newline at end of file From 22331d15cfb9de40248272ee5d76846ccc82a64c Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 13 Aug 2015 11:53:15 +0100 Subject: [PATCH 131/137] Propagate user exceptions thrown in DoFns. Support was added in Spark 1.5.0 for user exception propagation, see https://issues.apache.org/jira/browse/SPARK-8625. Fixes https://github.com/cloudera/spark-dataflow/issues/69 --- .../dataflow/spark/SparkPipelineRunner.java | 14 +++++-- .../dataflow/spark/SparkProcessContext.java | 10 ++++- .../dataflow/spark/SideEffectsTest.java | 37 +++++++++++++++---- 3 files changed, 49 insertions(+), 12 deletions(-) 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 index 792888dbeac5..5bed6e5f76df 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -115,10 +115,18 @@ public EvaluationResult run(Pipeline pipeline) { return ctxt; } catch (Exception e) { - // if the SparkException has a cause then wrap it in a RuntimeException - // (see https://issues.apache.org/jira/browse/SPARK-8625) + // 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) { - throw new RuntimeException(e.getCause()); + 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); 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 index ee2235aaecc4..e170926e554a 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -212,7 +212,7 @@ protected V computeNext() { try { doFn.processElement(SparkProcessContext.this); } catch (Exception e) { - throw new IllegalStateException(e); + throw new SparkProcessException(e); } outputIterator = getOutputIterator(); } else { @@ -223,7 +223,7 @@ protected V computeNext() { calledFinish = true; doFn.finishBundle(SparkProcessContext.this); } catch (Exception e) { - throw new IllegalStateException(e); + throw new SparkProcessException(e); } outputIterator = getOutputIterator(); continue; // try to consume outputIterator from start of loop @@ -234,4 +234,10 @@ protected V computeNext() { } } + static class SparkProcessException extends RuntimeException { + public SparkProcessException(Throwable t) { + super(t); + } + } + } 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 index 666737dc43ff..e1d5979775fa 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java @@ -20,14 +20,21 @@ 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 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(); @@ -36,19 +43,35 @@ public void test() throws Exception { pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - PCollection strings = pipeline.apply(Create.of("a")); - strings.apply(ParDo.of(new DoFn() { + pipeline.apply(Create.of("a")).apply(ParDo.of(new DoFn() { @Override public void processElement(ProcessContext c) throws Exception { - throw new IllegalStateException("Side effect"); + throw new UserException(); } })); try { pipeline.run(); fail("Run should thrown an exception"); - } catch (Exception e) { - // expected + } 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()).version(); + if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) { + assertTrue(e.getCause() instanceof UserException); + } } } -} \ No newline at end of file + + @Before + public void setup() { + System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "true"); + } + + @After + public void teardown() { + System.setProperty(SparkContextFactory.TEST_REUSE_SPARK_CONTEXT, "false"); + } +} From f930380ba03ad1f19d5bb1070823a5b403e6f93f Mon Sep 17 00:00:00 2001 From: Roberto Congiu Date: Mon, 23 Nov 2015 12:11:37 -0800 Subject: [PATCH 132/137] The example needs --inputFile, not --input, to designate the input file --- runners/spark/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index d3429fe1aaef..ccf85161fade 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -94,7 +94,7 @@ Then run the word count example using Spark submit with the `yarn-client` master --class com.google.cloud.dataflow.examples.WordCount \ --master yarn-client \ target/spark-dataflow-*-spark-app.jar \ - --input=kinglear.txt --output=out --runner=SparkPipelineRunner --sparkMaster=yarn-client + --inputFile=kinglear.txt --output=out --runner=SparkPipelineRunner --sparkMaster=yarn-client Check the output by running: From 7a2e9a72b0507efc6cb34c342a8d0983d63f1491 Mon Sep 17 00:00:00 2001 From: Amit Sela Date: Thu, 22 Oct 2015 17:41:54 +0300 Subject: [PATCH 133/137] Add spark-streaming support to spark-dataflow Add support for application name and streaming (default: false) Add pipeline options for streaming Add print output as an unbounded write Add default window strategy to represent Spark streaming micro-batches as fixed windows This translator helps to translate Dataflow transformations into Spark (+streaming) transformations. This will help to support streaming transformations separately Expose through the SparkPipelineTranslator Now Evaluator uses SparkPipelineTranslator to translate Add default application name StreamingEvaluation context to support DStream evaluation. Expose members and methods in EvaluationContext for inheritors Use configured app name in options A TransformTranslator for streaming Add support for spark streaming execution in the runner Fix comment Create input stream from a queue - mainly for testing I guess Add support to create input stream from queued values Override method to expose in package Test WordCount in streaming, just print out for now.. Stream print to console is a transformations of PCollection to PDone rename to CreateStream to differ from Dataflow Create It seems that in 1.3.1 short living streaming jobs fail (like unit tests). Maybe has something to do with SPARK-7930. fixed in 1.4.0 so bumped up. Expose some methods, add a method to check if RDDHolder exists make context final Streaming default should be local[1] to suppport unit tests No need for recurring context. Exposing additional parent methods. Added RUNNING state when stream is running. WordCount test runs 1 (sec) interval and compares to expected like in batch. Void Create triggers a no-input transformation transformations and output operations can be applied on streams/bounded collections in the pipeline foreachRDD is used for PDone transformation Commments SocketIO to consume stream from socket Comment Add support for Kafka input Comments and some patching-up Default is the same as in SparkPipelineOptions Adding licenses To satisfy license Javadoc and codestyle Satisfy license Javadoc and codestyle Check for DataflowAssertFailure because it won't propagate Since DataflowAssert doesn't propagate failures in streaming, use Aggregators to assert Use DataflowAssertStreaming Add kafka translation Embedded Kafka for unit test Kafka unit test import order license WindowingHelpers by Tom White @tomwhite Combine @tomwhite windowing branch into mine - values are windowed values now values are windowed values now Input is UNBOUNDED now Using windowing instead batchInterval to be determined by pipeline runner print the value not the windowed value remove support for for optimizations. for now. batchInterval is determined by the pipeline runner now Add streaming window pipeline visitor to determine windowing Add windowing support in streaming unit tests Combine.Globally is necessary so leave it fix line length renames Add implementation for GroupAlsoByWindow which helps to solve broken grouped/combinePerKey Line indentation unused codestyle Expose runtimeContext Make public Use the smallest window found (fixed/sliding) as the batch duration Make FieldGetter public Add support for windowing codestyle unused Update Spark to 1.5, kafka dependency should be provided Abstract Evaluator for common evaluator code. doVisitTransform per implementation. Added non-streaming windowing test by Tom White @tomwhite Fixed Combine.GroupedValues and Combine.Globally to work with WindowedValues without losing window properties. For now, Combine.PerKey is commented out until fixed to fully support WindowedValues. Support WindowedValues, Global or not, in Combine.PerKey After changes made to Combine.PerKey in 3a46150 it seems that the order has changed. Since ordere didn't seem relevant before the change, I don't see a reason not to change the expected value accordingly. Update Spark version to 1.5.2 --- runners/spark/pom.xml | 20 +- .../com/cloudera/dataflow/io/ConsoleIO.java | 60 +++ .../cloudera/dataflow/io/CreateStream.java | 66 +++ .../com/cloudera/dataflow/io/KafkaIO.java | 128 ++++++ .../cloudera/dataflow/spark/DoFnFunction.java | 21 +- .../dataflow/spark/EvaluationContext.java | 78 +++- .../dataflow/spark/MultiDoFnFunction.java | 31 +- .../dataflow/spark/SparkContextFactory.java | 10 +- .../spark/SparkPipelineEvaluator.java | 52 +++ .../dataflow/spark/SparkPipelineOptions.java | 13 +- .../dataflow/spark/SparkPipelineRunner.java | 108 +++-- .../spark/SparkPipelineTranslator.java | 27 ++ .../dataflow/spark/SparkProcessContext.java | 43 +- .../dataflow/spark/TransformTranslator.java | 274 ++++++++---- .../dataflow/spark/WindowingHelpers.java | 59 +++ .../SparkStreamingPipelineOptions.java | 40 ++ .../SparkStreamingPipelineOptionsFactory.java | 27 ++ ...parkStreamingPipelineOptionsRegistrar.java | 28 ++ .../streaming/StreamingEvaluationContext.java | 219 ++++++++++ .../StreamingTransformTranslator.java | 409 ++++++++++++++++++ .../StreamingWindowPipelineDetector.java | 99 +++++ ...aflow.sdk.options.PipelineOptionsRegistrar | 3 +- .../spark/MultiOutputWordCountTest.java | 2 +- .../dataflow/spark/SideEffectsTest.java | 2 +- .../dataflow/spark/SimpleWordCountTest.java | 2 +- .../dataflow/spark/WindowedWordCountTest.java | 63 +++ .../spark/streaming/KafkaStreamingTest.java | 133 ++++++ .../SimpleStreamingWordCountTest.java | 75 ++++ .../utils/DataflowAssertStreaming.java | 39 ++ .../streaming/utils/EmbeddedKafkaCluster.java | 315 ++++++++++++++ 30 files changed, 2255 insertions(+), 191 deletions(-) create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/io/ConsoleIO.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/io/CreateStream.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/io/KafkaIO.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineEvaluator.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/WindowingHelpers.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptions.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsFactory.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/SparkStreamingPipelineOptionsRegistrar.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java create mode 100644 runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/KafkaStreamingTest.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/DataflowAssertStreaming.java create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index df21c437dabc..5beb1c716890 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,7 +22,7 @@ License. UTF-8 UTF-8 1.7 - 1.3.1 + 1.5.2 1.0.0 @@ -281,6 +281,24 @@ License. ${spark.version} provided
+ + org.apache.spark + spark-streaming_2.10 + ${spark.version} + provided + + + org.apache.spark + spark-streaming-kafka_2.10 + ${spark.version} + provided + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + provided + com.google.guava guava 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 new file mode 100644 index 000000000000..bc19b39068a5 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 Unbound from() { + return new Unbound<>(10); + } + + public static Unbound from(int num) { + return new Unbound<>(num); + } + + public static class Unbound extends PTransform, PDone> { + + private final int num; + + Unbound(int num) { + this.num = num; + } + + public int getNum() { + return num; + } + + @Override + public PDone apply(PCollection input) { + return PDone.in(input.getPipeline()); + } + } + } +} 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 new file mode 100644 index 000000000000..9a9927873222 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 stream type + */ +public final class CreateStream { + + private CreateStream() { + } + + /** + * Define the input stream to create from queue. + * + * @param queuedValues defines the input stream + * @param stream type + * @return the queue that defines the input stream + */ + public static QueuedValues fromQueue(Iterable> queuedValues) { + return new QueuedValues<>(queuedValues); + } + + public static final class QueuedValues extends PTransform> { + + private final Iterable> queuedValues; + + QueuedValues(Iterable> queuedValues) { + Preconditions.checkNotNull(queuedValues, + "need to set the queuedValues of an Create.QueuedValues transform"); + this.queuedValues = queuedValues; + } + + public Iterable> getQueuedValues() { + return queuedValues; + } + + @Override + public PCollection apply(PInput input) { + // Spark streaming micro batches are bounded by default + return PCollection.createPrimitiveOutputInternal(input.getPipeline(), + WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); + } + } + +} 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 new file mode 100644 index 000000000000..154e6dacf37e --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 Kafka message key Class type + * @param Kafka message value Class type + * @return KafkaIO Unbound input + */ + public static Unbound from(Class> keyDecoder, + Class> valueDecoder, + Class key, + Class value, Set topics, + Map kafkaParams) { + return new Unbound<>(keyDecoder, valueDecoder, key, value, topics, kafkaParams); + } + + public static class Unbound extends PTransform>> { + + private final Class> keyDecoderClass; + private final Class> valueDecoderClass; + private final Class keyClass; + private final Class valueClass; + private final Set topics; + private final Map kafkaParams; + + Unbound(Class> keyDecoder, + Class> valueDecoder, Class key, + Class value, Set topics, Map 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> getKeyDecoderClass() { + return keyDecoderClass; + } + + public Class> getValueDecoderClass() { + return valueDecoderClass; + } + + public Class getValueClass() { + return valueClass; + } + + public Class getKeyClass() { + return keyClass; + } + + public Set getTopics() { + return topics; + } + + public Map getKafkaParams() { + return kafkaParams; + } + + @Override + public PCollection> apply(PInput input) { + // Spark streaming micro batches are bounded by default + return PCollection.createPrimitiveOutputInternal(input.getPipeline(), + WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); + } + } + + } +} 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 index 542f2ecc24b7..6617c56cb842 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java @@ -21,6 +21,7 @@ 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; @@ -30,7 +31,8 @@ * @param Input element type. * @param Output element type. */ -class DoFnFunction implements FlatMapFunction, O> { +public class DoFnFunction implements FlatMapFunction>, + WindowedValue> { private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -40,7 +42,7 @@ class DoFnFunction implements FlatMapFunction, O> { * @param runtime Runtime to apply function in. * @param sideInputs Side inputs used in DoFunction. */ - DoFnFunction(DoFn fn, + public DoFnFunction(DoFn fn, SparkRuntimeContext runtime, Map, BroadcastHelper> sideInputs) { this.mFunction = fn; @@ -49,16 +51,17 @@ class DoFnFunction implements FlatMapFunction, O> { } @Override - public Iterable call(Iterator iter) throws Exception { + public Iterable> call(Iterator> 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 { + private class ProcCtxt extends SparkProcessContext> { - private final List outputs = new LinkedList<>(); + private final List> outputs = new LinkedList<>(); ProcCtxt(DoFn fn, SparkRuntimeContext runtimeContext, Map, BroadcastHelper> sideInputs) { @@ -67,6 +70,12 @@ private class ProcCtxt extends SparkProcessContext { @Override public synchronized void output(O o) { + outputs.add(windowedValue != null ? windowedValue.withValue(o) : + WindowedValue.valueInEmptyWindows(o)); + } + + @Override + public synchronized void output(WindowedValue o) { outputs.add(o); } @@ -75,7 +84,7 @@ protected void clearOutput() { outputs.clear(); } - protected Iterator getOutputIterator() { + protected Iterator> getOutputIterator() { return outputs.iterator(); } } 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 index eb9554fd14b3..68e9d275ce49 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java @@ -55,8 +55,8 @@ public class EvaluationContext implements EvaluationResult { private final Set> leafRdds = new LinkedHashSet<>(); private final Set multireads = new LinkedHashSet<>(); private final Map pobjects = new LinkedHashMap<>(); - private final Map>> pview = new LinkedHashMap<>(); - private AppliedPTransform currentTransform; + private final Map>> pview = new LinkedHashMap<>(); + protected AppliedPTransform currentTransform; public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; @@ -76,21 +76,31 @@ private class RDDHolder { private Iterable values; private Coder coder; - private JavaRDDLike rdd; + private JavaRDDLike, ?> rdd; RDDHolder(Iterable values, Coder coder) { this.values = values; this.coder = coder; } - RDDHolder(JavaRDDLike rdd) { + RDDHolder(JavaRDDLike, ?> rdd) { this.rdd = rdd; } - public JavaRDDLike getRDD() { + public JavaRDDLike, ?> getRDD() { if (rdd == null) { - rdd = jsc.parallelize(CoderHelpers.toByteArrays(values, coder)) - .map(CoderHelpers.fromByteFunction(coder)); + Iterable> windowedValues = Iterables.transform(values, + new Function>() { + @Override + public WindowedValue apply(T t) { + // TODO: this is wrong if T is a TimestampedValue + return WindowedValue.valueInEmptyWindows(t); + } + }); + WindowedValue.ValueOnlyWindowedValueCoder windowCoder = + WindowedValue.getValueOnlyCoder(coder); + rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) + .map(CoderHelpers.fromByteFunction(windowCoder)); } return rdd; } @@ -98,7 +108,8 @@ private class RDDHolder { public Iterable getValues(PCollection pcollection) { if (values == null) { coder = pcollection.getCoder(); - JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(coder)); + JavaRDDLike bytesRDD = rdd.map(WindowingHelpers.unwindowFunction()) + .map(CoderHelpers.toByteFunction(coder)); List clientBytes = bytesRDD.collect(); values = Iterables.transform(clientBytes, new Function() { @Override @@ -109,25 +120,38 @@ public T apply(byte[] bytes) { } return values; } + + public Iterable> getWindowedValues(PCollection pcollection) { + return Iterables.transform(get(pcollection), new Function>() { + @Override + public WindowedValue apply(T t) { + return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place? + } + }); + } } - JavaSparkContext getSparkContext() { + protected JavaSparkContext getSparkContext() { return jsc; } - Pipeline getPipeline() { + protected Pipeline getPipeline() { return pipeline; } - SparkRuntimeContext getRuntimeContext() { + protected SparkRuntimeContext getRuntimeContext() { return runtime; } - void setCurrentTransform(AppliedPTransform transform) { + protected void setCurrentTransform(AppliedPTransform transform) { this.currentTransform = transform; } - I getInput(PTransform transform) { + protected AppliedPTransform getCurrentTransform() { + return currentTransform; + } + + protected I getInput(PTransform transform) { checkArgument(currentTransform != null && currentTransform.getTransform() == transform, "can only be called with current transform"); @SuppressWarnings("unchecked") @@ -135,7 +159,7 @@ I getInput(PTransform transform) { return input; } - O getOutput(PTransform transform) { + protected O getOutput(PTransform transform) { checkArgument(currentTransform != null && currentTransform.getTransform() == transform, "can only be called with current transform"); @SuppressWarnings("unchecked") @@ -143,20 +167,26 @@ O getOutput(PTransform transform) { return output; } - void setOutputRDD(PTransform transform, JavaRDDLike rdd) { + protected void setOutputRDD(PTransform transform, + JavaRDDLike, ?> rdd) { setRDD((PValue) getOutput(transform), rdd); } - void setOutputRDDFromValues(PTransform transform, Iterable values, + protected void setOutputRDDFromValues(PTransform transform, Iterable values, Coder coder) { pcollections.put((PValue) getOutput(transform), new RDDHolder<>(values, coder)); } - void setPView(PValue view, Iterable> value) { + void setPView(PValue view, Iterable> value) { pview.put(view, value); } - JavaRDDLike getRDD(PValue pvalue) { + protected boolean hasOutputRDD(PTransform 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); @@ -169,7 +199,7 @@ void setPView(PValue view, Iterable> value) { return rdd; } - void setRDD(PValue pvalue, JavaRDDLike rdd) { + protected void setRDD(PValue pvalue, JavaRDDLike, ?> rdd) { try { rdd.rdd().setName(pvalue.getName()); } catch (IllegalStateException e) { @@ -185,7 +215,7 @@ void setRDD(PValue pvalue, JavaRDDLike rdd) { } - Iterable> getPCollectionView(PCollectionView view) { + Iterable> getPCollectionView(PCollectionView view) { return pview.get(view); } @@ -194,7 +224,7 @@ Iterable> getPCollectionView(PCollectionView view) { * actions (like saving to a file) registered on them (i.e. they are performed for side * effects). */ - void computeOutputs() { + protected void computeOutputs() { for (RDDHolder rddHolder : leafRdds) { JavaRDDLike rdd = rddHolder.getRDD(); rdd.rdd().cache(); // cache so that any subsequent get() is cheap @@ -237,6 +267,12 @@ public Iterable get(PCollection pcollection) { return rddHolder.getValues(pcollection); } + Iterable> getWindowedValues(PCollection pcollection) { + @SuppressWarnings("unchecked") + RDDHolder rddHolder = (RDDHolder) pcollections.get(pcollection); + return rddHolder.getWindowedValues(pcollection); + } + @Override public void close() { SparkContextFactory.stopSparkContext(jsc); 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 index 8a9f8d53d7a9..17daff063cb8 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java @@ -19,6 +19,7 @@ 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; @@ -35,7 +36,8 @@ * @param Input type for DoFunction. * @param Output type for DoFunction. */ -class MultiDoFnFunction implements PairFlatMapFunction, TupleTag, Object> { +class MultiDoFnFunction + implements PairFlatMapFunction>, TupleTag, WindowedValue> { private final DoFn mFunction; private final SparkRuntimeContext mRuntimeContext; private final TupleTag mMainOutputTag; @@ -53,16 +55,17 @@ class MultiDoFnFunction implements PairFlatMapFunction, TupleT } @Override - public Iterable, Object>> call(Iterator iter) throws Exception { + public Iterable, WindowedValue>> + call(Iterator> 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, Object>> { + private class ProcCtxt extends SparkProcessContext, WindowedValue>> { - private final Multimap, Object> outputs = LinkedListMultimap.create(); + private final Multimap, WindowedValue> outputs = LinkedListMultimap.create(); ProcCtxt(DoFn fn, SparkRuntimeContext runtimeContext, Map, BroadcastHelper> sideInputs) { @@ -71,17 +74,23 @@ private class ProcCtxt extends SparkProcessContext, Obj @Override public synchronized void output(O o) { + outputs.put(mMainOutputTag, windowedValue.withValue(o)); + } + + @Override + public synchronized void output(WindowedValue o) { outputs.put(mMainOutputTag, o); } @Override public synchronized void sideOutput(TupleTag tag, T t) { - outputs.put(tag, t); + outputs.put(tag, windowedValue.withValue(t)); } @Override public void sideOutputWithTimestamp(TupleTag tupleTag, T t, Instant instant) { - outputs.put(tupleTag, t); + outputs.put(tupleTag, WindowedValue.of(t, instant, + windowedValue.getWindows(), windowedValue.getPane())); } @Override @@ -89,12 +98,14 @@ protected void clearOutput() { outputs.clear(); } - protected Iterator, Object>> getOutputIterator() { + protected Iterator, WindowedValue>> getOutputIterator() { return Iterators.transform(outputs.entries().iterator(), - new Function, Object>, Tuple2, Object>>() { + new Function, WindowedValue>, + Tuple2, WindowedValue>>() { @Override - public Tuple2, Object> apply(Map.Entry, Object> input) { - return new Tuple2, Object>(input.getKey(), input.getValue()); + public Tuple2, WindowedValue> apply(Map.Entry, + WindowedValue> input) { + return new Tuple2, WindowedValue>(input.getKey(), input.getValue()); } }); } 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 index b7570b39e233..97cbc20591dd 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java @@ -34,10 +34,10 @@ final class SparkContextFactory { private SparkContextFactory() { } - public static synchronized JavaSparkContext getSparkContext(String master) { + public static synchronized JavaSparkContext getSparkContext(String master, String appName) { if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) { if (sparkContext == null) { - sparkContext = createSparkContext(master); + sparkContext = createSparkContext(master, appName); sparkMaster = master; } else if (!master.equals(sparkMaster)) { throw new IllegalArgumentException(String.format("Cannot reuse spark context " + @@ -46,7 +46,7 @@ public static synchronized JavaSparkContext getSparkContext(String master) { } return sparkContext; } else { - return createSparkContext(master); + return createSparkContext(master, appName); } } @@ -56,10 +56,10 @@ public static synchronized void stopSparkContext(JavaSparkContext context) { } } - private static JavaSparkContext createSparkContext(String master) { + private static JavaSparkContext createSparkContext(String master, String appName) { SparkConf conf = new SparkConf(); conf.setMaster(master); - conf.setAppName("spark dataflow pipeline job"); + conf.setAppName(appName); conf.set("spark.serializer", KryoSerializer.class.getCanonicalName()); return new JavaSparkContext(conf); } 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 new file mode 100644 index 000000000000..67621801cf4c --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 > void doVisitTransform(TransformTreeNode + node) { + @SuppressWarnings("unchecked") + PT transform = (PT) node.getTransform(); + @SuppressWarnings("unchecked") + Class transformClass = (Class) (Class) transform.getClass(); + @SuppressWarnings("unchecked") TransformEvaluator evaluator = + (TransformEvaluator) translator.translate(transformClass); + LOG.info("Evaluating {}", transform); + AppliedPTransform appliedTransform = + AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform); + ctxt.setCurrentTransform(appliedTransform); + evaluator.evaluate(transform, ctxt); + ctxt.setCurrentTransform(null); + } +} 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 index 06793065649c..e96162ec3d03 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineOptions.java @@ -15,14 +15,25 @@ 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 { +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(); } 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 index 5bed6e5f76df..e980ae3e9a61 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java @@ -20,16 +20,23 @@ 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.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 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 @@ -50,6 +57,8 @@ * 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 { @@ -104,16 +113,49 @@ private SparkPipelineRunner(SparkPipelineOptions 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."); + final 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 " + + batchInterval.milliseconds() + "msec"); + 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(); - JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster()); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); - pipeline.traverseTopologically(new Evaluator(ctxt)); - ctxt.computeOutputs(); + 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."); + LOG.info("Pipeline execution complete."); - return ctxt; + 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 @@ -122,7 +164,7 @@ public EvaluationResult run(Pipeline pipeline) { // SparkProcessException), or just use the SparkException cause. if (e instanceof SparkException && e.getCause() != null) { if (e.getCause() instanceof SparkProcessContext.SparkProcessException && - e.getCause().getCause() != null) { + e.getCause().getCause() != null) { throw new RuntimeException(e.getCause().getCause()); } else { throw new RuntimeException(e.getCause()); @@ -133,21 +175,31 @@ public EvaluationResult run(Pipeline pipeline) { } } - private static final class Evaluator implements Pipeline.PipelineVisitor { + private EvaluationContext + createStreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline, + Duration batchDuration) { + SparkStreamingPipelineOptions streamingOptions = (SparkStreamingPipelineOptions) mOptions; + final 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; - private final EvaluationContext ctxt; + 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; - private Evaluator(EvaluationContext ctxt) { - this.ctxt = ctxt; - } - /** * 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 + * single + * TransformEvaluator; children nodes are ignored, and upon post-visiting the translated * composite node, the associated TransformEvaluator will be visited. */ private boolean inTranslatedCompositeNode() { @@ -160,10 +212,12 @@ public void enterCompositeTransform(TransformTreeNode node) { return; } + //noinspection unchecked if (node.getTransform() != null - && TransformTranslator.hasTransformEvaluator(node.getTransform().getClass())) { + && translator.hasTranslation( + (Class>) node.getTransform().getClass())) { LOG.info("Entering directly-translatable composite transform: '{}'", - node.getFullName()); + node.getFullName()); LOG.debug("Composite transform class: '{}'", node.getTransform().getClass()); currentTranslatedCompositeNode = node; } @@ -176,7 +230,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { // within the tree. if (inTranslatedCompositeNode() && node.equals(currentTranslatedCompositeNode)) { LOG.info("Post-visiting directly-translatable composite transform: '{}'", - node.getFullName()); + node.getFullName()); doVisitTransform(node); currentTranslatedCompositeNode = null; } @@ -191,23 +245,11 @@ public void visitTransform(TransformTreeNode node) { doVisitTransform(node); } - private > - void doVisitTransform(TransformTreeNode node) { - @SuppressWarnings("unchecked") - PT transform = (PT) node.getTransform(); - @SuppressWarnings("unchecked") - Class transformClass = (Class) (Class) transform.getClass(); - TransformEvaluator evaluator = TransformTranslator.getTransformEvaluator(transformClass); - LOG.info("Evaluating {}", transform); - AppliedPTransform appliedTransform = - AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform); - ctxt.setCurrentTransform(appliedTransform); - evaluator.evaluate(transform, ctxt); - ctxt.setCurrentTransform(null); - } + protected abstract > void + doVisitTransform(TransformTreeNode node); @Override - public void visitValue(PValue pvalue, TransformTreeNode node) { + public void visitValue(PValue value, TransformTreeNode producer) { } } } 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 new file mode 100644 index 000000000000..ff4931751e33 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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> clazz); + + TransformEvaluator> translate(Class> clazz); +} 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 index e170926e554a..f68efb4049bf 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java @@ -17,7 +17,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.Map; @@ -27,7 +26,6 @@ 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.GlobalWindow; 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; @@ -36,6 +34,8 @@ 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; @@ -44,18 +44,17 @@ abstract class SparkProcessContext extends DoFn.ProcessContext { private static final Logger LOG = LoggerFactory.getLogger(SparkProcessContext.class); - private static final Collection GLOBAL_WINDOWS = - Collections.singletonList(GlobalWindow.INSTANCE); - + private final DoFn fn; private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; - protected I element; + protected WindowedValue windowedValue; SparkProcessContext(DoFn fn, SparkRuntimeContext runtime, Map, BroadcastHelper> sideInputs) { fn.super(); + this.fn = fn; this.mRuntimeContext = runtime; this.mSideInputs = sideInputs; } @@ -81,6 +80,8 @@ public T sideInput(PCollectionView view) { @Override public abstract void output(O output); + public abstract void output(WindowedValue output); + @Override public void sideOutput(TupleTag tupleTag, T t) { String message = "sideOutput is an unsupported operation for doFunctions, use a " + @@ -107,27 +108,32 @@ public Aggregator createAggregatorInternal( @Override public I element() { - return element; + return windowedValue.getValue(); } @Override public void outputWithTimestamp(O output, Instant timestamp) { - output(output); + output(WindowedValue.of(output, timestamp, + windowedValue.getWindows(), windowedValue.getPane())); } @Override public Instant timestamp() { - return Instant.now(); + return windowedValue.getTimestamp(); } @Override public BoundedWindow window() { - return GlobalWindow.INSTANCE; + 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 PaneInfo.NO_FIRING; + return windowedValue.getPane(); } @Override @@ -136,13 +142,13 @@ public WindowingInternals windowingInternals() { @Override public Collection windows() { - return GLOBAL_WINDOWS; + return windowedValue.getWindows(); } @Override public void outputWindowedValue(O output, Instant timestamp, Collection windows, PaneInfo paneInfo) { - output(output); + output(WindowedValue.of(output, timestamp, windows, paneInfo)); } @Override @@ -159,7 +165,7 @@ public TimerInternals timerInternals() { @Override public PaneInfo pane() { - return PaneInfo.NO_FIRING; + return windowedValue.getPane(); } @Override @@ -174,7 +180,8 @@ public void writePCollectionViewData(TupleTag tag, protected abstract void clearOutput(); protected abstract Iterator getOutputIterator(); - protected Iterable getOutputIterable(final Iterator iter, final DoFn doFn) { + protected Iterable getOutputIterable(final Iterator> iter, + final DoFn doFn) { return new Iterable() { @Override public Iterator iterator() { @@ -185,12 +192,12 @@ public Iterator iterator() { private class ProcCtxtIterator extends AbstractIterator { - private final Iterator inputIterator; + private final Iterator> inputIterator; private final DoFn doFn; private Iterator outputIterator; private boolean calledFinish = false; - ProcCtxtIterator(Iterator iterator, DoFn doFn) { + ProcCtxtIterator(Iterator> iterator, DoFn doFn) { this.inputIterator = iterator; this.doFn = doFn; this.outputIterator = getOutputIterator(); @@ -208,7 +215,7 @@ protected V computeNext() { return outputIterator.next(); } else if (inputIterator.hasNext()) { clearOutput(); - element = inputIterator.next(); + windowedValue = inputIterator.next(); try { doFn.processElement(SparkProcessContext.this); } catch (Exception e) { 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 index 4537aa45284e..560d62f94cc4 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java @@ -30,12 +30,17 @@ 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; @@ -44,7 +49,7 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; + import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; @@ -77,10 +82,10 @@ public final class TransformTranslator { private TransformTranslator() { } - private static class FieldGetter { + public static class FieldGetter { private final Map fields; - FieldGetter(Class clazz) { + public FieldGetter(Class clazz) { this.fields = Maps.newHashMap(); for (Field f : clazz.getDeclaredFields()) { f.setAccessible(true); @@ -105,11 +110,11 @@ private static TransformEvaluator> flatten @Override public void evaluate(Flatten.FlattenPCollectionList transform, EvaluationContext context) { PCollectionList pcs = context.getInput(transform); - JavaRDD[] rdds = new JavaRDD[pcs.size()]; + JavaRDD>[] rdds = new JavaRDD[pcs.size()]; for (int i = 0; i < rdds.length; i++) { - rdds[i] = (JavaRDD) context.getRDD(pcs.get(i)); + rdds[i] = (JavaRDD>) context.getRDD(pcs.get(i)); } - JavaRDD rdd = context.getSparkContext().union(rdds); + JavaRDD> rdd = context.getSparkContext().union(rdds); context.setOutputRDD(transform, rdd); } }; @@ -120,8 +125,8 @@ private static TransformEvaluator> gbk() @Override public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContext context) { @SuppressWarnings("unchecked") - JavaRDDLike, ?> inRDD = - (JavaRDDLike, ?>) context.getInputRDD(transform); + JavaRDDLike>, ?> inRDD = + (JavaRDDLike>, ?>) context.getInputRDD(transform); @SuppressWarnings("unchecked") KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); Coder keyCoder = coder.getKeyCoder(); @@ -129,10 +134,13 @@ public void evaluate(GroupByKey.GroupByKeyOnly transform, EvaluationContex // Use coders to convert objects in the PCollection to byte arrays, so they // can be transferred over the network for the shuffle. - JavaRDDLike>, ?> outRDD = fromPair(toPair(inRDD) + JavaRDDLike>>, ?> outRDD = fromPair( + toPair(inRDD.map(WindowingHelpers.>unwindowFunction())) .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder)) .groupByKey() - .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder))); + .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder))) + // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK + .map(WindowingHelpers.>>windowFunction()); context.setOutputRDD(transform, outRDD); } }; @@ -144,11 +152,12 @@ private static TransformEvaluator> return new TransformEvaluator>() { @Override public void evaluate(Combine.GroupedValues transform, EvaluationContext context) { - Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); + Combine.KeyedCombineFn keyed = GROUPED_FG.get("fn", transform); @SuppressWarnings("unchecked") - JavaRDDLike>, ?> inRDD = - (JavaRDDLike>, ?>) context.getInputRDD(transform); - context.setOutputRDD(transform, inRDD.map(new KVFunction<>(keyed))); + JavaRDDLike>>, ?> inRDD = + (JavaRDDLike>>, ?>) context.getInputRDD(transform); + context.setOutputRDD(transform, + inRDD.map(new KVFunction<>(keyed))); } }; } @@ -163,7 +172,8 @@ public void evaluate(Combine.Globally transform, EvaluationContext context final Combine.CombineFn globally = COMBINE_GLOBALLY_FG.get("fn", transform); @SuppressWarnings("unchecked") - JavaRDDLike inRdd = (JavaRDDLike) context.getInputRDD(transform); + JavaRDDLike, ?> inRdd = + (JavaRDDLike, ?>) context.getInputRDD(transform); final Coder iCoder = context.getInput(transform).getCoder(); final Coder aCoder; @@ -176,7 +186,9 @@ public void evaluate(Combine.Globally transform, EvaluationContext context // Use coders to convert objects in the PCollection to byte arrays, so they // can be transferred over the network for the shuffle. - JavaRDD inRddBytes = inRdd.map(CoderHelpers.toByteFunction(iCoder)); + JavaRDD inRddBytes = inRdd + .map(WindowingHelpers.unwindowFunction()) + .map(CoderHelpers.toByteFunction(iCoder)); /*A*/ byte[] acc = inRddBytes.aggregate( CoderHelpers.toByteArray(globally.createAccumulator(), aCoder), @@ -206,7 +218,8 @@ public void evaluate(Combine.Globally transform, EvaluationContext context JavaRDD 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))); + context.setOutputRDD(transform, outRdd.map(CoderHelpers.fromByteFunction(coder)) + .map(WindowingHelpers.windowFunction())); } }; } @@ -220,14 +233,14 @@ public void evaluate(Combine.PerKey transform, EvaluationContext cont final Combine.KeyedCombineFn keyed = COMBINE_PERKEY_FG.get("fn", transform); @SuppressWarnings("unchecked") - JavaRDDLike, ?> inRdd = - (JavaRDDLike, ?>) context.getInputRDD(transform); + JavaRDDLike>, ?> inRdd = + (JavaRDDLike>, ?>) context.getInputRDD(transform); @SuppressWarnings("unchecked") KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); Coder keyCoder = inputCoder.getKeyCoder(); Coder viCoder = inputCoder.getValueCoder(); - Coder vaCoder = null; + Coder vaCoder; try { vaCoder = keyed.getAccumulatorCoder( context.getPipeline().getCoderRegistry(), keyCoder, viCoder); @@ -243,18 +256,35 @@ public void evaluate(Combine.PerKey transform, EvaluationContext cont // 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. - JavaPairRDD> inRddDuplicatedKeyPair = inRdd.mapToPair( - new PairFunction, K, KV>() { - @Override - public Tuple2> call(KV kv) { - return new Tuple2<>(kv.getKey(), kv); - } - }); + + // Key has to bw windowed in order to group by window as well + JavaPairRDD, WindowedValue>> inRddDuplicatedKeyPair = + inRdd.mapToPair( + new PairFunction>, WindowedValue, + WindowedValue>>() { + @Override + public Tuple2, + WindowedValue>> call(WindowedValue> kv) { + WindowedValue wk = WindowedValue.of(kv.getValue().getKey(), + kv.getTimestamp(), kv.getWindows(), kv.getPane()); + return new Tuple2<>(wk, kv); + } + }); + //-- windowed coders + final WindowedValue.FullWindowedValueCoder wkCoder = + WindowedValue.FullWindowedValueCoder.of(keyCoder, + context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder()); + final WindowedValue.FullWindowedValueCoder> wkviCoder = + WindowedValue.FullWindowedValueCoder.of(kviCoder, + context.getInput(transform).getWindowingStrategy().getWindowFn().windowCoder()); + final WindowedValue.FullWindowedValueCoder> 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 inRddDuplicatedKeyPairBytes = inRddDuplicatedKeyPair - .mapToPair(CoderHelpers.toByteFunction(keyCoder, kviCoder)); + .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 @@ -264,60 +294,85 @@ public Tuple2> call(KV kv) { new Function*/ byte[], /*KV*/ byte[]>() { @Override public /*KV*/ byte[] call(/*KV*/ byte[] input) { - KV kvi = CoderHelpers.fromByteArray(input, kviCoder); - VA va = keyed.createAccumulator(kvi.getKey()); - va = keyed.addInput(kvi.getKey(), va, kvi.getValue()); - return CoderHelpers.toByteArray(KV.of(kvi.getKey(), va), kvaCoder); + WindowedValue> wkvi = CoderHelpers.fromByteArray(input, wkviCoder); + VA va = keyed.createAccumulator(wkvi.getValue().getKey()); + va = keyed.addInput(wkvi.getValue().getKey(), va, wkvi.getValue().getValue()); + WindowedValue> wkva = + WindowedValue.of(KV.of(wkvi.getValue().getKey(), va), wkvi.getTimestamp(), + wkvi.getWindows(), wkvi.getPane()); + return CoderHelpers.toByteArray(wkva, wkvaCoder); } }, new Function2*/ byte[], /*KV*/ byte[], /*KV*/ byte[]>() { @Override public /*KV*/ byte[] call(/*KV*/ byte[] acc, /*KV*/ byte[] input) { - KV kva = CoderHelpers.fromByteArray(acc, kvaCoder); - KV kvi = CoderHelpers.fromByteArray(input, kviCoder); - VA va = keyed.addInput(kva.getKey(), kva.getValue(), kvi.getValue()); - kva = KV.of(kva.getKey(), va); - return CoderHelpers.toByteArray(KV.of(kva.getKey(), kva.getValue()), kvaCoder); + WindowedValue> wkva = CoderHelpers.fromByteArray(acc, wkvaCoder); + WindowedValue> 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*/ byte[], /*KV*/ byte[], /*KV*/ byte[]>() { @Override public /*KV*/ byte[] call(/*KV*/ byte[] acc1, /*KV*/ byte[] acc2) { - KV kva1 = CoderHelpers.fromByteArray(acc1, kvaCoder); - KV kva2 = CoderHelpers.fromByteArray(acc2, kvaCoder); - VA va = keyed.mergeAccumulators(kva1.getKey(), + WindowedValue> wkva1 = CoderHelpers.fromByteArray(acc1, wkvaCoder); + WindowedValue> 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(kva1.getValue(), kva2.getValue()))); - return CoderHelpers.toByteArray(KV.of(kva1.getKey(), va), kvaCoder); + Collections.unmodifiableList(Arrays.asList(wkva1.getValue().getValue(), + wkva2.getValue().getValue()))); + WindowedValue> wkva = WindowedValue.of(KV.of(wkva1.getValue().getKey(), + va), wkva1.getTimestamp(), wkva1.getWindows(), wkva1.getPane()); + return CoderHelpers.toByteArray(wkva, wkvaCoder); } }); - JavaPairRDD extracted = accumulatedBytes - .mapToPair(CoderHelpers.fromByteFunction(keyCoder, kvaCoder)) + JavaPairRDD, WindowedValue> extracted = accumulatedBytes + .mapToPair(CoderHelpers.fromByteFunction(wkCoder, wkvaCoder)) .mapValues( - new Function, VO>() { + new Function>, WindowedValue>() { @Override - public VO call(KV acc) { - return keyed.extractOutput(acc.getKey(), acc.getValue()); + public WindowedValue call(WindowedValue> acc) { + return WindowedValue.of(keyed.extractOutput(acc.getValue().getKey(), + acc.getValue().getValue()), acc.getTimestamp(), + acc.getWindows(), acc.getPane()); } }); - context.setOutputRDD(transform, fromPair(extracted)); + + context.setOutputRDD(transform, + fromPair(extracted) + .map(new Function, WindowedValue>, WindowedValue>>() { + @Override + public WindowedValue> call(KV, WindowedValue> kwvo) + throws Exception { + WindowedValue wvo = kwvo.getValue(); + KV kvo = KV.of(kwvo.getKey().getValue(), wvo.getValue()); + return WindowedValue.of(kvo, wvo.getTimestamp(), wvo.getWindows(), wvo.getPane()); + } + })); } }; } - private static final class KVFunction implements Function>, KV> { - private final Combine.KeyedCombineFn keyed; + private static final class KVFunction + implements Function>>, WindowedValue>> { + private final Combine.KeyedCombineFn keyed; - KVFunction(Combine.KeyedCombineFn keyed) { + KVFunction(Combine.KeyedCombineFn keyed) { this.keyed = keyed; } @Override - public KV call(KV> kv) throws Exception { - return KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())); + public WindowedValue> call(WindowedValue>> windowedKv) + throws Exception { + KV> kv = windowedKv.getValue(); + return WindowedValue.of(KV.of(kv.getKey(), keyed.apply(kv.getKey(), kv.getValue())), + windowedKv.getTimestamp(), windowedKv.getWindows(), windowedKv.getPane()); } } @@ -348,7 +403,8 @@ public void evaluate(ParDo.Bound transform, EvaluationContext context) { context.getRuntimeContext(), getSideInputs(transform.getSideInputs(), context)); @SuppressWarnings("unchecked") - JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); + JavaRDDLike, ?> inRDD = + (JavaRDDLike, ?>) context.getInputRDD(transform); context.setOutputRDD(transform, inRDD.mapPartitions(dofn)); } }; @@ -368,17 +424,21 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationContext context getSideInputs(transform.getSideInputs(), context)); @SuppressWarnings("unchecked") - JavaRDDLike inRDD = (JavaRDDLike) context.getInputRDD(transform); - JavaPairRDD, Object> all = inRDD + JavaRDDLike, ?> inRDD = + (JavaRDDLike, ?>) context.getInputRDD(transform); + JavaPairRDD, WindowedValue> all = inRDD .mapPartitionsToPair(multifn) .cache(); PCollectionTuple pct = context.getOutput(transform); for (Map.Entry, PCollection> e : pct.getAll().entrySet()) { @SuppressWarnings("unchecked") - JavaPairRDD, Object> filtered = + JavaPairRDD, WindowedValue> filtered = all.filter(new TupleTagFilter(e.getKey())); - context.setRDD(e.getValue(), filtered.values()); + @SuppressWarnings("unchecked") + // Object is the best we can do since different outputs can have different tags + JavaRDD> values = (JavaRDD) filtered.values(); + context.setRDD(e.getValue(), values); } } }; @@ -390,7 +450,8 @@ private static TransformEvaluator> readText() { @Override public void evaluate(TextIO.Read.Bound transform, EvaluationContext context) { String pattern = transform.getFilepattern(); - JavaRDD rdd = context.getSparkContext().textFile(pattern); + JavaRDD> rdd = context.getSparkContext().textFile(pattern) + .map(WindowingHelpers.windowFunction()); context.setOutputRDD(transform, rdd); } }; @@ -401,9 +462,11 @@ private static TransformEvaluator> writeText() { @Override public void evaluate(TextIO.Write.Bound transform, EvaluationContext context) { @SuppressWarnings("unchecked") - JavaPairRDD last = ((JavaRDDLike) context.getInputRDD(transform)) + JavaPairRDD last = + ((JavaRDDLike, ?>) context.getInputRDD(transform)) + .map(WindowingHelpers.unwindowFunction()) .mapToPair(new PairFunction() { + Void>() { @Override public Tuple2 call(T t) throws Exception { return new Tuple2<>(t, null); @@ -431,13 +494,13 @@ public void evaluate(AvroIO.Read.Bound transform, EvaluationContext context) AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, new Configuration()).keys(); - JavaRDD rdd = avroFile.map( + JavaRDD> rdd = avroFile.map( new Function, T>() { @Override public T call(AvroKey key) { return key.datum(); } - }); + }).map(WindowingHelpers.windowFunction()); context.setOutputRDD(transform, rdd); } }; @@ -456,7 +519,8 @@ public void evaluate(AvroIO.Write.Bound transform, EvaluationContext context) AvroJob.setOutputKeySchema(job, transform.getSchema()); @SuppressWarnings("unchecked") JavaPairRDD, NullWritable> last = - ((JavaRDDLike) context.getInputRDD(transform)) + ((JavaRDDLike, ?>) context.getInputRDD(transform)) + .map(WindowingHelpers.unwindowFunction()) .mapToPair(new PairFunction, NullWritable>() { @Override public Tuple2, NullWritable> call(T t) throws Exception { @@ -484,12 +548,13 @@ public void evaluate(HadoopIO.Read.Bound transform, EvaluationContext cont transform.getFormatClass(), transform.getKeyClass(), transform.getValueClass(), new Configuration()); - JavaRDD> rdd = file.map(new Function, KV>() { + JavaRDD>> rdd = + file.map(new Function, KV>() { @Override public KV call(Tuple2 t2) throws Exception { return KV.of(t2._1(), t2._2()); } - }); + }).map(WindowingHelpers.>windowFunction()); context.setOutputRDD(transform, rdd); } }; @@ -500,8 +565,9 @@ private static TransformEvaluator> writeHadoop @Override public void evaluate(HadoopIO.Write.Bound transform, EvaluationContext context) { @SuppressWarnings("unchecked") - JavaPairRDD last = ((JavaRDDLike, ?>) context + JavaPairRDD last = ((JavaRDDLike>, ?>) context .getInputRDD(transform)) + .map(WindowingHelpers.>unwindowFunction()) .mapToPair(new PairFunction, K, V>() { @Override public Tuple2 call(KV t) throws Exception { @@ -576,12 +642,25 @@ private static void writeHadoopFile(JavaPairRDD rdd, Configuration rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf); } - private static TransformEvaluator> window() { + static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class); + + private static TransformEvaluator> window() { return new TransformEvaluator>() { @Override public void evaluate(Window.Bound transform, EvaluationContext context) { - // TODO: detect and support non-global windows - context.setOutputRDD(transform, context.getInputRDD(transform)); + @SuppressWarnings("unchecked") + JavaRDDLike, ?> inRDD = + (JavaRDDLike, ?>) context.getInputRDD(transform); + WindowFn windowFn = WINDOW_FG.get("windowFn", transform); + if (windowFn instanceof GlobalWindows) { + context.setOutputRDD(transform, inRDD); + } else { + @SuppressWarnings("unchecked") + DoFn addWindowsDoFn = new AssignWindowsDoFn<>(windowFn); + DoFnFunction dofn = + new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null); + context.setOutputRDD(transform, inRDD.mapPartitions(dofn)); + } } }; } @@ -603,9 +682,9 @@ private static TransformEvaluator> viewAsSingleton() { return new TransformEvaluator>() { @Override public void evaluate(View.AsSingleton transform, EvaluationContext context) { - Iterable input = context.get(context.getInput(transform)); - context.setPView(context.getOutput(transform), Iterables.transform(input, - new WindowingFunction())); + Iterable> iter = + context.getWindowedValues(context.getInput(transform)); + context.setPView(context.getOutput(transform), iter); } }; } @@ -614,9 +693,9 @@ private static TransformEvaluator> viewAsIter() { return new TransformEvaluator>() { @Override public void evaluate(View.AsIterable transform, EvaluationContext context) { - Iterable input = context.get(context.getInput(transform)); - context.setPView(context.getOutput(transform), Iterables.transform(input, - new WindowingFunction())); + Iterable> iter = + context.getWindowedValues(context.getInput(transform)); + context.setPView(context.getOutput(transform), iter); } }; } @@ -625,23 +704,15 @@ private static TransformEvaluator> creat return new TransformEvaluator>() { @Override public void evaluate(View.CreatePCollectionView transform, EvaluationContext context) { - Iterable> iter = Iterables.transform( - context.get(context.getInput(transform)), new WindowingFunction()); + Iterable> iter = + context.getWindowedValues(context.getInput(transform)); context.setPView(context.getOutput(transform), iter); } }; } - private static class WindowingFunction implements com.google.common.base.Function> { - @Override - public WindowedValue apply(R t) { - return WindowedValue.valueInGlobalWindow(t); - } - } - private static final class TupleTagFilter - implements Function, Object>, Boolean> { + implements Function, WindowedValue>, Boolean> { private final TupleTag tag; @@ -650,7 +721,7 @@ private TupleTagFilter(TupleTag tag) { } @Override - public Boolean call(Tuple2, Object> input) { + public Boolean call(Tuple2, WindowedValue> input) { return tag.equals(input._1()); } } @@ -663,9 +734,11 @@ private static Map, BroadcastHelper> getSideInputs( } else { Map, BroadcastHelper> sideInputs = Maps.newHashMap(); for (PCollectionView view : views) { - Iterable> collectionView = context.getPCollectionView(view); + Iterable> collectionView = context.getPCollectionView(view); Coder>> coderInternal = view.getCoderInternal(); - BroadcastHelper helper = BroadcastHelper.create(collectionView, coderInternal); + @SuppressWarnings("unchecked") + BroadcastHelper helper = + BroadcastHelper.create((Iterable>) collectionView, coderInternal); //broadcast side inputs helper.broadcast(context.getSparkContext()); sideInputs.put(view.getTagInternal(), helper); @@ -702,8 +775,8 @@ private static Map, BroadcastHelper> getSideInputs( return EVALUATORS.containsKey(clazz); } - public static > TransformEvaluator getTransformEvaluator(Class - clazz) { + public static > TransformEvaluator + getTransformEvaluator(Class clazz) { @SuppressWarnings("unchecked") TransformEvaluator transform = (TransformEvaluator) EVALUATORS.get(clazz); if (transform == null) { @@ -711,4 +784,21 @@ private static Map, BroadcastHelper> getSideInputs( } return transform; } + + /** + * Translator matches Dataflow transformation with the appropriate evaluator. + */ + public static class Translator implements SparkPipelineTranslator { + + @Override + public boolean hasTranslation(Class> clazz) { + return hasTransformEvaluator(clazz); + } + + @Override + public TransformEvaluator> translate( + Class> clazz) { + return getTransformEvaluator(clazz); + } + } } 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 new file mode 100644 index 000000000000..90600b261f28 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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 The type of the object. + * @return A function that accepts an object and returns its {@link WindowedValue}. + */ + public static Function> windowFunction() { + return new Function>() { + @Override + public WindowedValue call(T t) { + return WindowedValue.valueInEmptyWindows(t); + } + }; + } + + /** + * A function for extracting the value from a {@link WindowedValue}. + * + * @param The type of the object. + * @return A function that accepts a {@link WindowedValue} and returns its value. + */ + public static Function, T> unwindowFunction() { + return new Function, T>() { + @Override + public T call(WindowedValue t) { + return t.getValue(); + } + }; + } +} 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 new file mode 100644 index 000000000000..57253f09b3ec --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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(); +} 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 new file mode 100644 index 000000000000..3b568af48b7e --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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); + } +} 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 new file mode 100644 index 000000000000..01c43751121a --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/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 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> getPipelineOptions() { + return ImmutableList.>of(SparkStreamingPipelineOptions + .class); + } +} 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 new file mode 100644 index 000000000000..5e1b42d42ee5 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingEvaluationContext.java @@ -0,0 +1,219 @@ +/* + * 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> pstreams = new LinkedHashMap<>(); + private final Set> 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 { + + private Iterable> values; + private Coder coder; + private JavaDStream> dStream; + + public DStreamHolder(Iterable> values, Coder coder) { + this.values = values; + this.coder = coder; + } + + public DStreamHolder(JavaDStream> dStream) { + this.dStream = dStream; + } + + @SuppressWarnings("unchecked") + public JavaDStream> getDStream() { + if (dStream == null) { + // create the DStream from values + Queue>> rddQueue = new LinkedBlockingQueue<>(); + for (Iterable v : values) { + setOutputRDDFromValues(currentTransform.getTransform(), v, coder); + rddQueue.offer((JavaRDD>) 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; + } + } + + public void setDStreamFromQueue(PTransform transform, Iterable> values, + Coder coder) { + pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder)); + } + + public , R>> + void setStream(PTransform transform, JavaDStreamLike, ?, R> dStream) { + PValue pvalue = (PValue) getOutput(transform); + @SuppressWarnings("unchecked") + DStreamHolder dStreamHolder = new DStreamHolder((JavaDStream) dStream); + pstreams.put(pvalue, dStreamHolder); + leafStreams.add(dStreamHolder); + } + + boolean hasStream(PTransform transform) { + PValue pvalue = (PValue) getInput(transform); + return pstreams.containsKey(pvalue); + } + + public JavaDStreamLike getStream(PTransform transform) { + PValue pvalue = (PValue) getInput(transform); + 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 + public void setInputRDD(PTransform transform, + JavaRDDLike, ?> rdd) { + setRDD((PValue) getInput(transform), rdd); + } + + // used to get the RDD transformation output and use it as the DStream transformation output + public JavaRDDLike getOutputRDD(PTransform transform) { + return getRDD((PValue) getOutput(transform)); + } + + public JavaStreamingContext getStreamingContext() { + return jssc; + } + + @Override + protected void computeOutputs() { + for (DStreamHolder streamHolder : leafStreams) { + @SuppressWarnings("unchecked") + JavaDStream> stream = (JavaDStream) streamHolder.getDStream(); + stream.foreachRDD(new Function>, Void>() { + @Override + public Void call(JavaRDD> 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 O getOutput(PTransform 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 void setOutputRDD(PTransform transform, + JavaRDDLike, ?> rdd) { + super.setOutputRDD(transform, rdd); + } + + @Override + protected void setOutputRDDFromValues(PTransform transform, Iterable values, + Coder coder) { + super.setOutputRDDFromValues(transform, values, coder); + } + + @Override + protected boolean hasOutputRDD(PTransform transform) { + return super.hasOutputRDD(transform); + } +} 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 new file mode 100644 index 000000000000..20ee88a1ec6a --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingTransformTranslator.java @@ -0,0 +1,409 @@ +/* + * 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.Map; +import java.util.Set; + +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.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 TransformEvaluator> print() { + return new TransformEvaluator>() { + @Override + public void evaluate(ConsoleIO.Write.Unbound transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaDStreamLike, ?, JavaRDD>> dstream = + (JavaDStreamLike, ?, JavaRDD>>) + ((StreamingEvaluationContext) context).getStream(transform); + dstream.map(WindowingHelpers.unwindowFunction()) + .print(transform.getNum()); + } + }; + } + + private static TransformEvaluator> kafka() { + return new TransformEvaluator>() { + @Override + public void evaluate(KafkaIO.Read.Unbound transform, EvaluationContext context) { + JavaStreamingContext jssc = ((StreamingEvaluationContext) context).getStreamingContext(); + Class keyClazz = transform.getKeyClass(); + Class valueClazz = transform.getValueClass(); + Class> keyDecoderClazz = transform.getKeyDecoderClass(); + Class> valueDecoderClazz = transform.getValueDecoderClass(); + Map kafkaParams = transform.getKafkaParams(); + Set topics = transform.getTopics(); + JavaPairInputDStream inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz, + valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics); + JavaDStream>> inputStream = + inputPairStream.map(new Function, KV>() { + @Override + public KV call(Tuple2 t2) throws Exception { + return KV.of(t2._1(), t2._2()); + } + }).map(WindowingHelpers.>windowFunction()); + ((StreamingEvaluationContext) context).setStream(transform, inputStream); + } + }; + } + + private static TransformEvaluator> + create() { + return new TransformEvaluator>() { + @SuppressWarnings("unchecked") + @Override + public void evaluate(com.google.cloud.dataflow.sdk.transforms.Create.Values + transform, EvaluationContext context) { + Iterable elems = transform.getElements(); + Coder coder = ((StreamingEvaluationContext) context).getOutput(transform) + .getCoder(); + if (coder != VoidCoder.of()) { + // actual create + ((StreamingEvaluationContext) context).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 + ((StreamingEvaluationContext) context).setDStreamFromQueue(transform, + Collections.>singletonList(Collections.singletonList((Void) null)), + (Coder) coder); + } + } + }; + } + + private static TransformEvaluator> createFromQueue() { + return new TransformEvaluator>() { + @Override + public void evaluate(CreateStream.QueuedValues transform, EvaluationContext + context) { + Iterable> values = transform.getQueuedValues(); + Coder coder = ((StreamingEvaluationContext) context).getOutput(transform) + .getCoder(); + ((StreamingEvaluationContext) context).setDStreamFromQueue(transform, values, + coder); + } + }; + } + + private static > TransformEvaluator rddTransform( + final SparkPipelineTranslator rddTranslator) { + return new TransformEvaluator() { + @SuppressWarnings("unchecked") + @Override + public void evaluate(final PT transform, + final EvaluationContext context) { + final TransformEvaluator rddEvaluator = + rddTranslator.translate((Class>) transform.getClass()); + + if (((StreamingEvaluationContext) context).hasStream(transform)) { + JavaDStreamLike, ?, JavaRDD>> dStream = + (JavaDStreamLike, ?, JavaRDD>>) + ((StreamingEvaluationContext) context).getStream(transform); + + ((StreamingEvaluationContext) context).setStream(transform, dStream + .transform(new RDDTransform<>((StreamingEvaluationContext) context, + 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 PTransform type + */ + private static final class RDDTransform> + implements Function>, JavaRDD>> { + + private final StreamingEvaluationContext context; + private final AppliedPTransform appliedPTransform; + private final TransformEvaluator rddEvaluator; + private final PT transform; + + + private RDDTransform(StreamingEvaluationContext context, TransformEvaluator rddEvaluator, + PT transform) { + this.context = context; + this.appliedPTransform = context.getCurrentTransform(); + this.rddEvaluator = rddEvaluator; + this.transform = transform; + } + + @Override + @SuppressWarnings("unchecked") + public JavaRDD> + call(JavaRDD> 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().>emptyRDD()); + } + JavaRDD> outRDD = + (JavaRDD>) context.getOutputRDD(transform); + context.setCurrentTransform(existingAPT); + return outRDD; + } + } + + @SuppressWarnings("unchecked") + private static > TransformEvaluator foreachRDD( + final SparkPipelineTranslator rddTranslator) { + return new TransformEvaluator() { + @Override + public void evaluate(final PT transform, + final EvaluationContext context) { + final TransformEvaluator rddEvaluator = + rddTranslator.translate((Class>) transform.getClass()); + + if (((StreamingEvaluationContext) context).hasStream(transform)) { + JavaDStreamLike, ?, JavaRDD>> dStream = + (JavaDStreamLike, ?, JavaRDD>>) ( + (StreamingEvaluationContext) context).getStream(transform); + + dStream.foreachRDD(new RDDOutputOperator<>((StreamingEvaluationContext) context, + rddEvaluator, transform)); + } else { + rddEvaluator.evaluate(transform, context); + } + } + }; + } + + /** + * RDD output function. + * + * @param PTransform type + */ + private static final class RDDOutputOperator> + implements Function>, Void> { + + private final StreamingEvaluationContext context; + private final AppliedPTransform appliedPTransform; + private final TransformEvaluator rddEvaluator; + private final PT transform; + + + private RDDOutputOperator(StreamingEvaluationContext context, TransformEvaluator rddEvaluator, + PT transform) { + this.context = context; + this.appliedPTransform = context.getCurrentTransform(); + this.rddEvaluator = rddEvaluator; + this.transform = transform; + } + + @Override + @SuppressWarnings("unchecked") + public Void call(JavaRDD> rdd) throws Exception { + AppliedPTransform existingAPT = context.getCurrentTransform(); + context.setCurrentTransform(appliedPTransform); + context.setInputRDD(transform, rdd); + rddEvaluator.evaluate(transform, context); + context.setCurrentTransform(existingAPT); + return null; + } + } + + static final TransformTranslator.FieldGetter WINDOW_FG = + new TransformTranslator.FieldGetter(Window.Bound.class); + + private static TransformEvaluator> window() { + return new TransformEvaluator>() { + @Override + public void evaluate(Window.Bound transform, EvaluationContext context) { + //--- first we apply windowing to the stream + WindowFn windowFn = WINDOW_FG.get("windowFn", transform); + @SuppressWarnings("unchecked") + JavaDStream> dStream = + (JavaDStream>) + ((StreamingEvaluationContext) context).getStream(transform); + if (windowFn instanceof FixedWindows) { + Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize() + .getMillis()); + ((StreamingEvaluationContext) context) + .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()); + ((StreamingEvaluationContext) context) + .setStream(transform, dStream.window(windowDuration, slideDuration)); + } + //--- then we apply windowing to the elements + DoFn addWindowsDoFn = new AssignWindowsDoFn<>(windowFn); + DoFnFunction dofn = new DoFnFunction<>(addWindowsDoFn, + ((StreamingEvaluationContext)context).getRuntimeContext(), null); + @SuppressWarnings("unchecked") + JavaDStreamLike, ?, JavaRDD>> dstream = + (JavaDStreamLike, ?, JavaRDD>>) + ((StreamingEvaluationContext) context).getStream(transform); + //noinspection unchecked + ((StreamingEvaluationContext) context).setStream(transform, + dstream.mapPartitions(dofn)); + } + }; + } + + private static final Map, 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()); + } + + private static final Set> UNSUPPORTTED_EVALUATORS = Sets + .newHashSet(); + + static { + //TODO - add support for the following + UNSUPPORTTED_EVALUATORS.add(TextIO.Read.Bound.class); + UNSUPPORTTED_EVALUATORS.add(TextIO.Write.Bound.class); + UNSUPPORTTED_EVALUATORS.add(AvroIO.Read.Bound.class); + UNSUPPORTTED_EVALUATORS.add(AvroIO.Write.Bound.class); + UNSUPPORTTED_EVALUATORS.add(HadoopIO.Read.Bound.class); + UNSUPPORTTED_EVALUATORS.add(HadoopIO.Write.Bound.class); + UNSUPPORTTED_EVALUATORS.add(Flatten.FlattenPCollectionList.class); + } + + private static > boolean hasTransformEvaluator(Class clazz) { + return EVALUATORS.containsKey(clazz); + } + + @SuppressWarnings("unchecked") + private static > TransformEvaluator + getTransformEvaluator(Class clazz, SparkPipelineTranslator rddTranslator) { + TransformEvaluator transform = (TransformEvaluator) EVALUATORS.get(clazz); + if (transform == null) { + if (UNSUPPORTTED_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 PTranform's Input and Output class + Class pTOutputClazz = getPTransformOutputClazz(clazz); + if (pTOutputClazz == PDone.class) { + return foreachRDD(rddTranslator); + } else { + return rddTransform(rddTranslator); + } + } + return transform; + } + + private static > Class + getPTransformOutputClazz(Class 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> clazz) { + // streaming includes rdd transformations as well + return hasTransformEvaluator(clazz) || rddTranslator.hasTranslation(clazz); + } + + @Override + public TransformEvaluator> translate( + Class> clazz) { + return getTransformEvaluator(clazz, rddTranslator); + } + } +} 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 new file mode 100644 index 000000000000..f9b2d2b75af8 --- /dev/null +++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/streaming/StreamingWindowPipelineDetector.java @@ -0,0 +1,99 @@ +/* + * 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); + } + + 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 + protected > void + doVisitTransform(TransformTreeNode node) { + @SuppressWarnings("unchecked") + PT transform = (PT) node.getTransform(); + @SuppressWarnings("unchecked") + Class transformClass = (Class) (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; + } +} 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 045d5dd0cfb1..5733a8658523 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,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. # -com.cloudera.dataflow.spark.SparkPipelineOptionsRegistrar \ No newline at end of file +com.cloudera.dataflow.spark.SparkPipelineOptionsRegistrar +com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptionsRegistrar \ No newline at end of file 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 index 179816d73e96..2df8493e1701 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/MultiOutputWordCountTest.java @@ -65,7 +65,7 @@ public void testRun() throws Exception { EvaluationResult res = SparkPipelineRunner.create().run(p); Iterable> actualLower = res.get(luc.get(lowerCnts)); - Assert.assertEquals("and", actualLower.iterator().next().getKey()); + Assert.assertEquals("are", actualLower.iterator().next().getKey()); Iterable> actualUpper = res.get(luc.get(upperCnts)); Assert.assertEquals("Here", actualUpper.iterator().next().getKey()); Iterable actualUniqCount = res.get(unique); 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 index e1d5979775fa..ce7acda5aa44 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SideEffectsTest.java @@ -58,7 +58,7 @@ public void processElement(ProcessContext c) throws Exception { // 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()).version(); + String version = SparkContextFactory.getSparkContext(options.getSparkMaster(), options.getAppName()).version(); if (!version.startsWith("1.3.") && !version.startsWith("1.4.")) { assertTrue(e.getCause() instanceof UserException); } 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 index 0f6db1f7ccb4..3d85f467e7f1 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java @@ -93,7 +93,7 @@ public void processElement(ProcessContext c) { } } - private static class CountWords extends PTransform, PCollection> { + public static class CountWords extends PTransform, PCollection> { @Override public PCollection apply(PCollection lines) { 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 new file mode 100644 index 000000000000..c16878eef32b --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/WindowedWordCountTest.java @@ -0,0 +1,63 @@ +/* + * 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 WORDS = Arrays.asList(WORDS_ARRAY); + private static final List TIMESTAMPS = Arrays.asList(TIMESTAMPS_ARRAY); + private static final List 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 inputWords = p.apply(Create.timestamped(WORDS, TIMESTAMPS)) + .setCoder(StringUtf8Coder.of()); + PCollection windowedWords = inputWords + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))); + + PCollection output = windowedWords.apply(new SimpleWordCountTest.CountWords()); + + DataflowAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + EvaluationResult res = SparkPipelineRunner.create().run(p); + res.close(); + } + +} 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 new file mode 100644 index 000000000000..8778e004ad5c --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/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 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 KAFKA_MESSAGES = ImmutableMap.of( + "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4" + ); + private static final Set EXPECTED = ImmutableSet.of( + "k1,v1", "k2,v2", "k3,v3", "k4,v4" + ); + private final static long TEST_TIMEOUT_MSEC = 1000L; + + @BeforeClass + public static void init() throws IOException, InterruptedException { + 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 stringSerializer = new StringSerializer(); + @SuppressWarnings("unchecked") KafkaProducer kafkaProducer = + new KafkaProducer(producerProps, stringSerializer, stringSerializer); + for (Map.Entry en : KAFKA_MESSAGES.entrySet()) { + kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue())); + } + kafkaProducer.close(); + } + + @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 kafkaParams = ImmutableMap.of( + "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(), + "auto.offset.reset", "smallest" + ); + + PCollection> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class, + StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC), + kafkaParams)); + PCollection> windowedWords = kafkaInput + .apply(Window.>into(FixedWindows.of(Duration.standardSeconds(1)))); + + PCollection formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn())); + + DataflowAssert.thatIterable(formattedKV.apply(View.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, String> { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey() + "," + c.element().getValue()); + } + } + +} 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 new file mode 100644 index 000000000000..613e517a0eaa --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/SimpleStreamingWordCountTest.java @@ -0,0 +1,75 @@ +/* + * 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.ConsoleIO; +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.Assert; +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> WORDS_QUEUE = + Collections.>singletonList(Arrays.asList(WORDS_ARRAY)); + private static final Set EXPECTED_COUNT_SET = + ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); + final static 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 inputWords = + p.apply(CreateStream.fromQueue(WORDS_QUEUE)).setCoder(StringUtf8Coder.of()); + PCollection windowedWords = inputWords + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(1)))); + + PCollection output = windowedWords.apply(new SimpleWordCountTest.CountWords()); + + DataflowAssert.thatIterable(output.apply(View.asIterable())) + .containsInAnyOrder(EXPECTED_COUNT_SET); + + EvaluationResult res = SparkPipelineRunner.create(options).run(p); + res.close(); + + DataflowAssertStreaming.assertNoFailures(res); + } +} \ No newline at end of file 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 new file mode 100644 index 000000000000..c0c5976e2714 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/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 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); + } +} 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 new file mode 100644 index 000000000000..6daae54c0f60 --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/utils/EmbeddedKafkaCluster.java @@ -0,0 +1,315 @@ +/* + * 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; + +/** + * https://gist.github.com/fjavieralba/7930018 + */ +public class EmbeddedKafkaCluster { + private final List ports; + private final String zkConnection; + private final Properties baseProperties; + + private final String brokerList; + + private final List brokers; + private final List 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 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 List resolvePorts(List ports) { + List resolvedPorts = new ArrayList(); + for (Integer port : ports) { + resolvedPorts.add(resolvePort(port)); + } + return resolvedPorts; + } + + private int resolvePort(int port) { + if (port == -1) { + return TestUtils.getAvailablePort(); + } + return port; + } + + private String constructBrokerList(List 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 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 getPorts() { + return ports; + } + + public String getZkConnection() { + return zkConnection; + } + + public void shutdown() { + for (KafkaServer broker : brokers) { + try { + broker.shutdown(); + } catch (Exception e) { + e.printStackTrace(); + } + } + for (File logDir : logDirs) { + try { + TestUtils.deleteFile(logDir); + } catch (FileNotFoundException e) { + e.printStackTrace(); + } + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("EmbeddedKafkaCluster{"); + sb.append("brokerList='").append(brokerList).append('\''); + sb.append('}'); + return sb.toString(); + } + + 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 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("embeeded-zk/snapshot"); + this.logDir = TestUtils.constructTempDir("embeeded-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() { + final StringBuilder sb = new StringBuilder("EmbeddedZookeeper{"); + sb.append("connection=").append(getConnection()); + sb.append('}'); + return sb.toString(); + } + } + + static class SystemTime implements Time { + public long milliseconds() { + return System.currentTimeMillis(); + } + + public long nanoseconds() { + return System.nanoTime(); + } + + public void sleep(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + // Ignore + } + } + } + + static class TestUtils { + private static final Random RANDOM = new Random(); + + private TestUtils() { + } + + public 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; + } + + public static int getAvailablePort() { + try { + ServerSocket socket = new ServerSocket(0); + try { + return socket.getLocalPort(); + } finally { + socket.close(); + } + } catch (IOException e) { + throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e); + } + } + + public 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(); + } + } +} From 34787303499499b5b4e8d616e0ec71784940b44e Mon Sep 17 00:00:00 2001 From: Sela Date: Sat, 16 Jan 2016 13:00:11 +0200 Subject: [PATCH 134/137] Add support for Flattenning (union) PCollections and test Wrong packcage utils --- .../streaming/StreamingEvaluationContext.java | 8 +- .../StreamingTransformTranslator.java | 86 +++++++++++-------- .../spark/streaming/FlattenStreamingTest.java | 86 +++++++++++++++++++ .../SimpleStreamingWordCountTest.java | 2 - 4 files changed, 144 insertions(+), 38 deletions(-) create mode 100644 runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.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 index 5e1b42d42ee5..3290729d281b 100644 --- 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 @@ -116,7 +116,10 @@ boolean hasStream(PTransform transform) { } public JavaDStreamLike getStream(PTransform transform) { - PValue pvalue = (PValue) getInput(transform); + return getStream((PValue) getInput(transform)); + } + + public JavaDStreamLike getStream(PValue pvalue) { DStreamHolder dStreamHolder = pstreams.get(pvalue); JavaDStreamLike dStream = dStreamHolder.getDStream(); leafStreams.remove(dStreamHolder); @@ -175,6 +178,9 @@ public State getState() { } //---------------- override in order to expose in package + protected I getInput(PTransform transform) { + return super.getInput(transform); + } @Override protected O getOutput(PTransform transform) { return super.getOutput(transform); 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 index 20ee88a1ec6a..0153f388d3fd 100644 --- 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 @@ -17,9 +17,11 @@ 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; @@ -40,6 +42,7 @@ 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; @@ -83,8 +86,7 @@ public void evaluate(ConsoleIO.Write.Unbound transform, EvaluationContext contex JavaDStreamLike, ?, JavaRDD>> dstream = (JavaDStreamLike, ?, JavaRDD>>) ((StreamingEvaluationContext) context).getStream(transform); - dstream.map(WindowingHelpers.unwindowFunction()) - .print(transform.getNum()); + dstream.map(WindowingHelpers.unwindowFunction()).print(transform.getNum()); } }; } @@ -93,7 +95,8 @@ private static TransformEvaluator> kafka() { return new TransformEvaluator>() { @Override public void evaluate(KafkaIO.Read.Unbound transform, EvaluationContext context) { - JavaStreamingContext jssc = ((StreamingEvaluationContext) context).getStreamingContext(); + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; + JavaStreamingContext jssc = sec.getStreamingContext(); Class keyClazz = transform.getKeyClass(); Class valueClazz = transform.getValueClass(); Class> keyDecoderClazz = transform.getKeyDecoderClass(); @@ -109,7 +112,7 @@ public KV call(Tuple2 t2) throws Exception { return KV.of(t2._1(), t2._2()); } }).map(WindowingHelpers.>windowFunction()); - ((StreamingEvaluationContext) context).setStream(transform, inputStream); + sec.setStream(transform, inputStream); } }; } @@ -121,19 +124,18 @@ public KV call(Tuple2 t2) throws Exception { @Override public void evaluate(com.google.cloud.dataflow.sdk.transforms.Create.Values transform, EvaluationContext context) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; Iterable elems = transform.getElements(); - Coder coder = ((StreamingEvaluationContext) context).getOutput(transform) - .getCoder(); + Coder coder = sec.getOutput(transform).getCoder(); if (coder != VoidCoder.of()) { // actual create - ((StreamingEvaluationContext) context).setOutputRDDFromValues(transform, - elems, coder); + 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 - ((StreamingEvaluationContext) context).setDStreamFromQueue(transform, + sec.setDStreamFromQueue(transform, Collections.>singletonList(Collections.singletonList((Void) null)), (Coder) coder); } @@ -144,13 +146,30 @@ public void evaluate(com.google.cloud.dataflow.sdk.transforms.Create.Values private static TransformEvaluator> createFromQueue() { return new TransformEvaluator>() { @Override - public void evaluate(CreateStream.QueuedValues transform, EvaluationContext - context) { + public void evaluate(CreateStream.QueuedValues transform, EvaluationContext context) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; Iterable> values = transform.getQueuedValues(); - Coder coder = ((StreamingEvaluationContext) context).getOutput(transform) - .getCoder(); - ((StreamingEvaluationContext) context).setDStreamFromQueue(transform, values, - coder); + Coder coder = sec.getOutput(transform).getCoder(); + sec.setDStreamFromQueue(transform, values, coder); + } + }; + } + + private static TransformEvaluator> flattenPColl() { + return new TransformEvaluator>() { + @SuppressWarnings("unchecked") + @Override + public void evaluate(Flatten.FlattenPCollectionList transform, EvaluationContext context) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; + PCollectionList pcs = sec.getInput(transform); + JavaDStream> first = + (JavaDStream>) sec.getStream(pcs.get(0)); + List>> rest = Lists.newArrayListWithCapacity(pcs.size() - 1); + for (int i = 1; i < pcs.size(); i++) { + rest.add((JavaDStream>) sec.getStream(pcs.get(i))); + } + JavaDStream> dstream = sec.getStreamingContext().union(first, rest); + sec.setStream(transform, dstream); } }; } @@ -165,14 +184,14 @@ public void evaluate(final PT transform, final TransformEvaluator rddEvaluator = rddTranslator.translate((Class>) transform.getClass()); - if (((StreamingEvaluationContext) context).hasStream(transform)) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; + if (sec.hasStream(transform)) { JavaDStreamLike, ?, JavaRDD>> dStream = (JavaDStreamLike, ?, JavaRDD>>) - ((StreamingEvaluationContext) context).getStream(transform); + sec.getStream(transform); - ((StreamingEvaluationContext) context).setStream(transform, dStream - .transform(new RDDTransform<>((StreamingEvaluationContext) context, - rddEvaluator, 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 @@ -235,13 +254,13 @@ public void evaluate(final PT transform, final TransformEvaluator rddEvaluator = rddTranslator.translate((Class>) transform.getClass()); - if (((StreamingEvaluationContext) context).hasStream(transform)) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; + if (sec.hasStream(transform)) { JavaDStreamLike, ?, JavaRDD>> dStream = - (JavaDStreamLike, ?, JavaRDD>>) ( - (StreamingEvaluationContext) context).getStream(transform); + (JavaDStreamLike, ?, JavaRDD>>) + sec.getStream(transform); - dStream.foreachRDD(new RDDOutputOperator<>((StreamingEvaluationContext) context, - rddEvaluator, transform)); + dStream.foreachRDD(new RDDOutputOperator<>(sec, rddEvaluator, transform)); } else { rddEvaluator.evaluate(transform, context); } @@ -290,24 +309,22 @@ private static TransformEvaluator> return new TransformEvaluator>() { @Override public void evaluate(Window.Bound transform, EvaluationContext context) { + StreamingEvaluationContext sec = (StreamingEvaluationContext) context; //--- first we apply windowing to the stream WindowFn windowFn = WINDOW_FG.get("windowFn", transform); @SuppressWarnings("unchecked") JavaDStream> dStream = - (JavaDStream>) - ((StreamingEvaluationContext) context).getStream(transform); + (JavaDStream>) sec.getStream(transform); if (windowFn instanceof FixedWindows) { Duration windowDuration = Durations.milliseconds(((FixedWindows) windowFn).getSize() .getMillis()); - ((StreamingEvaluationContext) context) - .setStream(transform, dStream.window(windowDuration)); + 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()); - ((StreamingEvaluationContext) context) - .setStream(transform, dStream.window(windowDuration, slideDuration)); + sec.setStream(transform, dStream.window(windowDuration, slideDuration)); } //--- then we apply windowing to the elements DoFn addWindowsDoFn = new AssignWindowsDoFn<>(windowFn); @@ -316,10 +333,9 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { @SuppressWarnings("unchecked") JavaDStreamLike, ?, JavaRDD>> dstream = (JavaDStreamLike, ?, JavaRDD>>) - ((StreamingEvaluationContext) context).getStream(transform); + sec.getStream(transform); //noinspection unchecked - ((StreamingEvaluationContext) context).setStream(transform, - dstream.mapPartitions(dofn)); + sec.setStream(transform, dstream.mapPartitions(dofn)); } }; } @@ -333,6 +349,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { 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> UNSUPPORTTED_EVALUATORS = Sets @@ -346,7 +363,6 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { UNSUPPORTTED_EVALUATORS.add(AvroIO.Write.Bound.class); UNSUPPORTTED_EVALUATORS.add(HadoopIO.Read.Bound.class); UNSUPPORTTED_EVALUATORS.add(HadoopIO.Write.Bound.class); - UNSUPPORTTED_EVALUATORS.add(Flatten.FlattenPCollectionList.class); } private static > boolean hasTransformEvaluator(Class clazz) { 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 new file mode 100644 index 000000000000..d818e9a5146f --- /dev/null +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/streaming/FlattenStreamingTest.java @@ -0,0 +1,86 @@ +/* + * 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.SparkStreamingPipelineOptions; +import com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptionsFactory; +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> WORDS_QUEUE_1 = + Collections.>singletonList(Arrays.asList(WORDS_ARRAY_1)); + private static final String[] WORDS_ARRAY_2 = { + "five", "six", "seven", "eight"}; + private static final List> WORDS_QUEUE_2 = + Collections.>singletonList(Arrays.asList(WORDS_ARRAY_2)); + private static final String[] EXPECTED_UNION = { + "one", "two", "three", "four", "five", "six", "seven", "eight"}; + final static 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 w1 = + p.apply(CreateStream.fromQueue(WORDS_QUEUE_1)).setCoder(StringUtf8Coder.of()); + PCollection windowedW1 = + w1.apply(Window.into(FixedWindows.of(Duration.standardSeconds(1)))); + PCollection w2 = + p.apply(CreateStream.fromQueue(WORDS_QUEUE_2)).setCoder(StringUtf8Coder.of()); + PCollection windowedW2 = + w2.apply(Window.into(FixedWindows.of(Duration.standardSeconds(1)))); + PCollectionList list = PCollectionList.of(windowedW1).and(windowedW2); + PCollection union = list.apply(Flatten.pCollections()); + + DataflowAssert.thatIterable(union.apply(View.asIterable())) + .containsInAnyOrder(EXPECTED_UNION); + + EvaluationResult res = SparkPipelineRunner.create(options).run(p); + res.close(); + + DataflowAssertStreaming.assertNoFailures(res); + } + +} 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 index 613e517a0eaa..eb23b5a32d55 100644 --- 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 @@ -23,7 +23,6 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.collect.ImmutableSet; -import com.cloudera.dataflow.io.ConsoleIO; import com.cloudera.dataflow.io.CreateStream; import com.cloudera.dataflow.spark.EvaluationResult; import com.cloudera.dataflow.spark.SimpleWordCountTest; @@ -31,7 +30,6 @@ import com.cloudera.dataflow.spark.streaming.utils.DataflowAssertStreaming; import org.joda.time.Duration; -import org.junit.Assert; import org.junit.Test; import java.util.Arrays; From a9168bfc9e759b6b22e4ac1c116f86b4389bf17d Mon Sep 17 00:00:00 2001 From: Tom White Date: Thu, 21 Jan 2016 08:51:57 +0000 Subject: [PATCH 135/137] Upgrade to latest SDK version 1.3.0 --- runners/spark/pom.xml | 2 +- .../test/java/com/cloudera/dataflow/spark/NumShardsTest.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 5beb1c716890..115ec71c80a0 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -23,7 +23,7 @@ License. UTF-8 1.7 1.5.2 - 1.0.0 + 1.3.0
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 index 9572b0f65392..6849389d84e7 100644 --- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java +++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/NumShardsTest.java @@ -20,6 +20,7 @@ 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; @@ -64,7 +65,7 @@ public void testText() throws Exception { Pipeline p = Pipeline.create(options); PCollection inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); PCollection output = inputWords.apply(new WordCount.CountWords()) - .apply(ParDo.of(new WordCount.FormatAsTextFn())); + .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(); From 89a21ca9ed04f625e4ccbd2f2142f395f7f9979a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 21 Jan 2016 15:25:49 +0100 Subject: [PATCH 136/137] Try to clean up some build warnings, related to generics, and try to further untangle some generics issues. Update plugins. Fix some minor code issues from inspection. --- runners/spark/pom.xml | 12 ++-- .../cloudera/dataflow/hadoop/HadoopIO.java | 2 +- .../dataflow/hadoop/WritableCoder.java | 3 +- .../dataflow/spark/BroadcastHelper.java | 8 +-- .../cloudera/dataflow/spark/CoderHelpers.java | 2 +- .../cloudera/dataflow/spark/DoFnFunction.java | 1 + .../dataflow/spark/EvaluationContext.java | 11 ++- .../dataflow/spark/EvaluationResult.java | 4 +- .../dataflow/spark/MultiDoFnFunction.java | 1 + .../dataflow/spark/SparkContextFactory.java | 10 +-- .../dataflow/spark/SparkPipelineRunner.java | 28 ++++---- .../spark/SparkPipelineTranslator.java | 2 +- .../dataflow/spark/SparkProcessContext.java | 4 +- .../dataflow/spark/TransformTranslator.java | 20 +++--- .../streaming/StreamingEvaluationContext.java | 49 ++++++------- .../StreamingTransformTranslator.java | 71 ++++++++----------- .../StreamingWindowPipelineDetector.java | 3 +- .../dataflow/spark/CombineGloballyTest.java | 6 +- .../spark/HadoopFileFormatPipelineTest.java | 20 ++---- .../dataflow/spark/SerializationTest.java | 5 +- .../dataflow/spark/SimpleWordCountTest.java | 5 +- .../spark/TransformTranslatorTest.java | 2 +- .../spark/streaming/FlattenStreamingTest.java | 4 +- .../spark/streaming/KafkaStreamingTest.java | 14 ++-- .../SimpleStreamingWordCountTest.java | 2 +- .../streaming/utils/EmbeddedKafkaCluster.java | 55 +++++++------- 26 files changed, 162 insertions(+), 182 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 115ec71c80a0..399e9e77ad34 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -32,7 +32,7 @@ License. org.apache.maven.plugins maven-compiler-plugin - 3.3 + 3.5 ${java.version} ${java.version} @@ -44,7 +44,7 @@ License. org.apache.maven.plugins maven-surefire-plugin - 2.18.1 + 2.19.1 1 false @@ -53,7 +53,7 @@ License. org.apache.maven.plugins maven-checkstyle-plugin - 2.15 + 2.17 build-resources/header-file.txt build-resources/checkstyle.xml @@ -100,7 +100,7 @@ License. org.apache.maven.plugins maven-clean-plugin - 2.6.1 + 3.0.0 org.apache.maven.plugins @@ -133,7 +133,7 @@ License. org.apache.maven.plugins maven-release-plugin - 2.5.2 + 2.5.3 org.apache.maven.scm @@ -224,7 +224,7 @@ License. org.apache.maven.plugins maven-shade-plugin - 2.4.1 + 2.4.3 package 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 index 6389db368fad..c79f2113cc51 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/HadoopIO.java @@ -99,7 +99,7 @@ public static final class Write { private Write() { } - public static Bound to(String filenamePrefix, + public static Bound to(String filenamePrefix, Class> format, Class key, Class value) { return new Bound<>(filenamePrefix, format, key, value); } 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 index 759fb581ef6b..324b203a2f15 100644 --- a/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java +++ b/runners/spark/src/main/java/com/cloudera/dataflow/hadoop/WritableCoder.java @@ -33,8 +33,7 @@ import org.apache.hadoop.io.Writable; /** - * A {@code WritableCoder} is a {@link com.google.cloud.dataflow.sdk.coders.Coder} for a - * Java class that implements {@link org.apache.hadoop.io.Writable}. + * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. * *

To use, specify the coder type on a PCollection: *

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
index 6ef70f328183..8dca939c5247 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/BroadcastHelper.java
@@ -28,10 +28,10 @@
 abstract class BroadcastHelper implements Serializable {
 
   /**
-   * If the property dataflow.spark.directBroadcast is set to
-   * true then Spark serialization (Kryo) will be used to broadcast values
+   * 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 com.google.cloud.dataflow.sdk.coders.Coder}.
+   * the appropriate {@link Coder}.
    */
   public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast";
 
@@ -78,7 +78,7 @@ public void broadcast(JavaSparkContext jsc) {
 
   /**
    * A {@link com.cloudera.dataflow.spark.BroadcastHelper} that uses a
-   * {@link com.google.cloud.dataflow.sdk.coders.Coder} to encode values as byte arrays
+   * {@link Coder} to encode values as byte arrays
    * before broadcasting.
    * @param 
    */
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
index b5e86b6f2198..0ae06c1a9e49 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/CoderHelpers.java
@@ -159,7 +159,7 @@ public Tuple2 call(Tuple2 tuple) {
 
   /**
    * A function wrapper for converting a byte array pair to a key-value pair, where
-   * values are Iterable.
+   * values are {@link Iterable}.
    *
    * @param keyCoder Coder to deserialize keys.
    * @param valueCoder Coder to deserialize values.
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
index 6617c56cb842..2bcfec3dfc43 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/DoFnFunction.java
@@ -84,6 +84,7 @@ protected void clearOutput() {
       outputs.clear();
     }
 
+    @Override
     protected Iterator> getOutputIterator() {
       return outputs.iterator();
     }
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
index 68e9d275ce49..356acabbc1ee 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
@@ -25,7 +25,6 @@
 
 import com.google.cloud.dataflow.sdk.Pipeline;
 import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
 import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
 import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
 import com.google.cloud.dataflow.sdk.transforms.Aggregator;
@@ -50,7 +49,7 @@ public class EvaluationContext implements EvaluationResult {
   private final JavaSparkContext jsc;
   private final Pipeline pipeline;
   private final SparkRuntimeContext runtime;
-  private final CoderRegistry registry;
+  //private final CoderRegistry registry;
   private final Map> pcollections = new LinkedHashMap<>();
   private final Set> leafRdds = new LinkedHashSet<>();
   private final Set multireads = new LinkedHashSet<>();
@@ -61,7 +60,7 @@ public class EvaluationContext implements EvaluationResult {
   public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
     this.jsc = jsc;
     this.pipeline = pipeline;
-    this.registry = pipeline.getCoderRegistry();
+    //this.registry = pipeline.getCoderRegistry();
     this.runtime = new SparkRuntimeContext(jsc, pipeline);
   }
 
@@ -87,7 +86,7 @@ private class RDDHolder {
       this.rdd = rdd;
     }
 
-    public JavaRDDLike, ?> getRDD() {
+    JavaRDDLike, ?> getRDD() {
       if (rdd == null) {
         Iterable> windowedValues = Iterables.transform(values,
             new Function>() {
@@ -105,7 +104,7 @@ public WindowedValue apply(T t) {
       return rdd;
     }
 
-    public Iterable getValues(PCollection pcollection) {
+    Iterable getValues(PCollection pcollection) {
       if (values == null) {
         coder = pcollection.getCoder();
         JavaRDDLike bytesRDD = rdd.map(WindowingHelpers.unwindowFunction())
@@ -121,7 +120,7 @@ public T apply(byte[] bytes) {
       return values;
     }
 
-    public Iterable> getWindowedValues(PCollection pcollection) {
+    Iterable> getWindowedValues(PCollection pcollection) {
       return Iterables.transform(get(pcollection), new Function>() {
         @Override
         public WindowedValue apply(T t) {
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
index b40d44978707..aad029aec17d 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationResult.java
@@ -36,7 +36,7 @@ public interface EvaluationResult extends PipelineResult {
   /**
    * Retrieve an object of Type T associated with the PValue passed in.
    *
-   * @param pval PValue to retireve associated data for.
+   * @param pval PValue to retrieve associated data for.
    * @param   Type of object to return.
    * @return Native object.
    */
@@ -55,7 +55,7 @@ public interface EvaluationResult extends PipelineResult {
   /**
    * Releases any runtime resources, including distributed-execution contexts currently held by
    * this EvaluationResult; once close() has been called,
-   * {@link com.cloudera.dataflow.spark.EvaluationResult#get(PCollection)} might
+   * {@link EvaluationResult#get(PCollection)} might
    * not work for subsequent calls.
    */
   void close();
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
index 17daff063cb8..d269788bfda3 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/MultiDoFnFunction.java
@@ -98,6 +98,7 @@ protected void clearOutput() {
       outputs.clear();
     }
 
+    @Override
     protected Iterator, WindowedValue>> getOutputIterator() {
       return Iterators.transform(outputs.entries().iterator(),
           new Function, WindowedValue>,
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
index 97cbc20591dd..d3e8c9b1cabf 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkContextFactory.java
@@ -22,11 +22,11 @@
 final class SparkContextFactory {
 
   /**
-   * If the property dataflow.spark.test.reuseSparkContext is set to
-   * true then the Spark context will be reused for dataflow pipelines.
+   * 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.
    */
-  public static final String TEST_REUSE_SPARK_CONTEXT =
+  static final String TEST_REUSE_SPARK_CONTEXT =
       "dataflow.spark.test.reuseSparkContext";
   private static JavaSparkContext sparkContext;
   private static String sparkMaster;
@@ -34,7 +34,7 @@ final class SparkContextFactory {
   private SparkContextFactory() {
   }
 
-  public static synchronized JavaSparkContext getSparkContext(String master, String appName) {
+  static synchronized JavaSparkContext getSparkContext(String master, String appName) {
     if (Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
       if (sparkContext == null) {
         sparkContext = createSparkContext(master, appName);
@@ -50,7 +50,7 @@ public static synchronized JavaSparkContext getSparkContext(String master, Strin
     }
   }
 
-  public static synchronized void stopSparkContext(JavaSparkContext context) {
+  static synchronized void stopSparkContext(JavaSparkContext context) {
     if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) {
       context.stop();
     }
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
index e980ae3e9a61..5287f2011732 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
@@ -120,7 +120,7 @@ public EvaluationResult run(Pipeline pipeline) {
             mOptions.getClass().getSimpleName());
       }
       LOG.info("Executing pipeline using the SparkPipelineRunner.");
-      final JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions
+      JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions
               .getSparkMaster(), mOptions.getAppName());
 
       if (mOptions.isStreaming()) {
@@ -135,8 +135,7 @@ public EvaluationResult run(Pipeline pipeline) {
         }
 
         Duration batchInterval = streamingWindowPipelineDetector.getBatchDuration();
-        LOG.info("Setting Spark streaming batchInterval to " +
-            batchInterval.milliseconds() + "msec");
+        LOG.info("Setting Spark streaming batchInterval to {} msec", batchInterval.milliseconds());
         EvaluationContext ctxt = createStreamingEvaluationContext(jsc, pipeline, batchInterval);
 
         pipeline.traverseTopologically(new SparkPipelineEvaluator(ctxt, translator));
@@ -179,7 +178,7 @@ public EvaluationResult run(Pipeline pipeline) {
       createStreamingEvaluationContext(JavaSparkContext jsc, Pipeline pipeline,
       Duration batchDuration) {
     SparkStreamingPipelineOptions streamingOptions = (SparkStreamingPipelineOptions) mOptions;
-    final JavaStreamingContext jssc = new JavaStreamingContext(jsc, batchDuration);
+    JavaStreamingContext jssc = new JavaStreamingContext(jsc, batchDuration);
     return new StreamingEvaluationContext(jsc, pipeline, jssc, streamingOptions.getTimeout());
   }
 
@@ -208,18 +207,15 @@ private boolean inTranslatedCompositeNode() {
 
     @Override
     public void enterCompositeTransform(TransformTreeNode node) {
-      if (inTranslatedCompositeNode()) {
-        return;
-      }
-
-      //noinspection unchecked
-      if (node.getTransform() != null
-              && translator.hasTranslation(
-              (Class>) node.getTransform().getClass())) {
-        LOG.info("Entering directly-translatable composite transform: '{}'",
-                node.getFullName());
-        LOG.debug("Composite transform class: '{}'", node.getTransform().getClass());
-        currentTranslatedCompositeNode = node;
+      if (!inTranslatedCompositeNode() && node.getTransform() != null) {
+        @SuppressWarnings("unchecked")
+        Class> transformClass =
+            (Class>) node.getTransform().getClass();
+        if (translator.hasTranslation(transformClass)) {
+          LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName());
+          LOG.debug("Composite transform class: '{}'", node.getTransform().getClass());
+          currentTranslatedCompositeNode = node;
+        }
       }
     }
 
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
index ff4931751e33..d90363f43c74 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineTranslator.java
@@ -23,5 +23,5 @@ public interface SparkPipelineTranslator {
 
   boolean hasTranslation(Class> clazz);
 
-  TransformEvaluator> translate(Class> clazz);
+  > TransformEvaluator translate(Class clazz);
 }
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
index f68efb4049bf..73cec25e2489 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkProcessContext.java
@@ -195,7 +195,7 @@ private class ProcCtxtIterator extends AbstractIterator {
     private final Iterator> inputIterator;
     private final DoFn doFn;
     private Iterator outputIterator;
-    private boolean calledFinish = false;
+    private boolean calledFinish;
 
     ProcCtxtIterator(Iterator> iterator, DoFn doFn) {
       this.inputIterator = iterator;
@@ -242,7 +242,7 @@ protected V computeNext() {
   }
 
   static class SparkProcessException extends RuntimeException {
-    public SparkProcessException(Throwable t) {
+    SparkProcessException(Throwable t) {
       super(t);
     }
   }
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
index 560d62f94cc4..db335edbd89a 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
@@ -247,8 +247,8 @@ public void evaluate(Combine.PerKey transform, EvaluationContext cont
         } catch (CannotProvideCoderException e) {
           throw new IllegalStateException("Could not determine coder for accumulator", e);
         }
-        final Coder> kviCoder = KvCoder.of(keyCoder, viCoder);
-        final Coder> kvaCoder = KvCoder.of(keyCoder, vaCoder);
+        Coder> kviCoder = KvCoder.of(keyCoder, viCoder);
+        Coder> 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
@@ -437,7 +437,8 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationContext context
               all.filter(new TupleTagFilter(e.getKey()));
           @SuppressWarnings("unchecked")
           // Object is the best we can do since different outputs can have different tags
-          JavaRDD> values = (JavaRDD) filtered.values();
+          JavaRDD> values =
+              (JavaRDD>) (JavaRDD) filtered.values();
           context.setRDD(e.getValue(), values);
         }
       }
@@ -602,19 +603,19 @@ private ShardTemplateInformation(int numShards, String shardTemplate, String
       this.filenameSuffix = filenameSuffix;
     }
 
-    public int getNumShards() {
+    int getNumShards() {
       return numShards;
     }
 
-    public String getShardTemplate() {
+    String getShardTemplate() {
       return shardTemplate;
     }
 
-    public String getFilenamePrefix() {
+    String getFilenamePrefix() {
       return filenamePrefix;
     }
 
-    public String getFilenameSuffix() {
+    String getFilenameSuffix() {
       return filenameSuffix;
     }
   }
@@ -642,7 +643,7 @@ private static  void writeHadoopFile(JavaPairRDD rdd, Configuration
     rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
   }
 
-  static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
+  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
 
   private static  TransformEvaluator> window() {
     return new TransformEvaluator>() {
@@ -796,8 +797,7 @@ public boolean hasTranslation(Class> clazz) {
     }
 
     @Override
-    public TransformEvaluator> translate(
-        Class> clazz) {
+    public > TransformEvaluator translate(Class clazz) {
       return getTransformEvaluator(clazz);
     }
   }
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
index 3290729d281b..5ecd56258414 100644
--- 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
@@ -70,17 +70,17 @@ private class DStreamHolder {
     private Coder coder;
     private JavaDStream> dStream;
 
-    public DStreamHolder(Iterable> values, Coder coder) {
+    DStreamHolder(Iterable> values, Coder coder) {
       this.values = values;
       this.coder = coder;
     }
 
-    public DStreamHolder(JavaDStream> dStream) {
+    DStreamHolder(JavaDStream> dStream) {
       this.dStream = dStream;
     }
 
     @SuppressWarnings("unchecked")
-    public JavaDStream> getDStream() {
+    JavaDStream> getDStream() {
       if (dStream == null) {
         // create the DStream from values
         Queue>> rddQueue = new LinkedBlockingQueue<>();
@@ -96,16 +96,14 @@ public JavaDStream> getDStream() {
     }
   }
 
-  public  void setDStreamFromQueue(PTransform transform, Iterable> values,
-      Coder coder) {
+   void setDStreamFromQueue(
+      PTransform transform, Iterable> values, Coder coder) {
     pstreams.put((PValue) getOutput(transform), new DStreamHolder<>(values, coder));
   }
 
-  public , R>>
-      void setStream(PTransform transform, JavaDStreamLike, ?, R> dStream) {
+   void setStream(PTransform transform, JavaDStream> dStream) {
     PValue pvalue = (PValue) getOutput(transform);
-    @SuppressWarnings("unchecked")
-    DStreamHolder dStreamHolder = new DStreamHolder((JavaDStream) dStream);
+    DStreamHolder dStreamHolder = new DStreamHolder<>(dStream);
     pstreams.put(pvalue, dStreamHolder);
     leafStreams.add(dStreamHolder);
   }
@@ -115,11 +113,11 @@ boolean hasStream(PTransform transform) {
     return pstreams.containsKey(pvalue);
   }
 
-  public JavaDStreamLike getStream(PTransform transform) {
+  JavaDStreamLike getStream(PTransform transform) {
     return getStream((PValue) getInput(transform));
   }
 
-  public JavaDStreamLike getStream(PValue pvalue) {
+  JavaDStreamLike getStream(PValue pvalue) {
     DStreamHolder dStreamHolder = pstreams.get(pvalue);
     JavaDStreamLike dStream = dStreamHolder.getDStream();
     leafStreams.remove(dStreamHolder);
@@ -127,13 +125,13 @@ boolean hasStream(PTransform transform) {
   }
 
   // used to set the RDD from the DStream in the RDDHolder for transformation
-  public  void setInputRDD(PTransform transform,
-      JavaRDDLike, ?> rdd) {
+   void setInputRDD(
+      PTransform transform, JavaRDDLike, ?> rdd) {
     setRDD((PValue) getInput(transform), rdd);
   }
 
   // used to get the RDD transformation output and use it as the DStream transformation output
-  public JavaRDDLike getOutputRDD(PTransform transform) {
+  JavaRDDLike getOutputRDD(PTransform transform) {
     return getRDD((PValue) getOutput(transform));
   }
 
@@ -144,19 +142,21 @@ public JavaStreamingContext getStreamingContext() {
   @Override
   protected void computeOutputs() {
     for (DStreamHolder streamHolder : leafStreams) {
-      @SuppressWarnings("unchecked")
-      JavaDStream> stream = (JavaDStream) streamHolder.getDStream();
-      stream.foreachRDD(new Function>, Void>() {
-        @Override
-        public Void call(JavaRDD> rdd) throws Exception {
-          rdd.rdd().cache();
-          rdd.count();
-          return null;
-        }
-      }); // force a DStream action
+      computeOutput(streamHolder);
     }
   }
 
+  private static  void computeOutput(DStreamHolder streamHolder) {
+    streamHolder.getDStream().foreachRDD(new Function>, Void>() {
+      @Override
+      public Void call(JavaRDD> rdd) throws Exception {
+        rdd.rdd().cache();
+        rdd.count();
+        return null;
+      }
+    }); // force a DStream action
+  }
+
   @Override
   public void close() {
     if (timeout > 0) {
@@ -178,6 +178,7 @@ public State getState() {
   }
 
   //---------------- override in order to expose in package
+  @Override
   protected  I getInput(PTransform transform) {
     return super.getInput(transform);
   }
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
index 0153f388d3fd..d8ae5e899dfa 100644
--- 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
@@ -81,7 +81,7 @@ private StreamingTransformTranslator() {
   private static  TransformEvaluator> print() {
     return new TransformEvaluator>() {
       @Override
-      public void evaluate(ConsoleIO.Write.Unbound transform, EvaluationContext context) {
+      public void evaluate(ConsoleIO.Write.Unbound transform, EvaluationContext context) {
         @SuppressWarnings("unchecked")
         JavaDStreamLike, ?, JavaRDD>> dstream =
             (JavaDStreamLike, ?, JavaRDD>>)
@@ -117,13 +117,11 @@ public KV call(Tuple2 t2) throws Exception {
     };
   }
 
-  private static  TransformEvaluator>
-      create() {
-    return new TransformEvaluator>() {
+  private static  TransformEvaluator> create() {
+    return new TransformEvaluator>() {
       @SuppressWarnings("unchecked")
       @Override
-      public void evaluate(com.google.cloud.dataflow.sdk.transforms.Create.Values
-                                   transform, EvaluationContext context) {
+      public void evaluate(Create.Values transform, EvaluationContext context) {
         StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
         Iterable elems = transform.getElements();
         Coder coder = sec.getOutput(transform).getCoder();
@@ -179,10 +177,9 @@ public void evaluate(Flatten.FlattenPCollectionList transform, EvaluationCont
     return new TransformEvaluator() {
       @SuppressWarnings("unchecked")
       @Override
-      public void evaluate(final PT transform,
-                           final EvaluationContext context) {
-        final TransformEvaluator rddEvaluator =
-            rddTranslator.translate((Class>) transform.getClass());
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator rddEvaluator =
+            rddTranslator.translate((Class) transform.getClass());
 
         StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
         if (sec.hasStream(transform)) {
@@ -212,11 +209,11 @@ private static final class RDDTransform>
 
     private final StreamingEvaluationContext context;
     private final AppliedPTransform appliedPTransform;
-    private final TransformEvaluator rddEvaluator;
+    private final TransformEvaluator rddEvaluator;
     private final PT transform;
 
 
-    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator rddEvaluator,
+    private RDDTransform(StreamingEvaluationContext context, TransformEvaluator rddEvaluator,
         PT transform) {
       this.context = context;
       this.appliedPTransform = context.getCurrentTransform();
@@ -249,10 +246,9 @@ private RDDTransform(StreamingEvaluationContext context, TransformEvaluator rddE
       final SparkPipelineTranslator rddTranslator) {
     return new TransformEvaluator() {
       @Override
-      public void evaluate(final PT transform,
-                           final EvaluationContext context) {
-        final TransformEvaluator rddEvaluator =
-            rddTranslator.translate((Class>) transform.getClass());
+      public void evaluate(PT transform, EvaluationContext context) {
+        TransformEvaluator rddEvaluator =
+            rddTranslator.translate((Class) transform.getClass());
 
         StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
         if (sec.hasStream(transform)) {
@@ -278,12 +274,12 @@ private static final class RDDOutputOperator>
 
     private final StreamingEvaluationContext context;
     private final AppliedPTransform appliedPTransform;
-    private final TransformEvaluator rddEvaluator;
+    private final TransformEvaluator rddEvaluator;
     private final PT transform;
 
 
-    private RDDOutputOperator(StreamingEvaluationContext context, TransformEvaluator rddEvaluator,
-        PT transform) {
+    private RDDOutputOperator(StreamingEvaluationContext context,
+        TransformEvaluator rddEvaluator, PT transform) {
       this.context = context;
       this.appliedPTransform = context.getCurrentTransform();
       this.rddEvaluator = rddEvaluator;
@@ -302,7 +298,7 @@ public Void call(JavaRDD> rdd) throws Exception {
     }
   }
 
-  static final TransformTranslator.FieldGetter WINDOW_FG =
+  private static final TransformTranslator.FieldGetter WINDOW_FG =
       new TransformTranslator.FieldGetter(Window.Bound.class);
 
   private static  TransformEvaluator> window() {
@@ -334,7 +330,6 @@ public void evaluate(Window.Bound transform, EvaluationContext context) {
         JavaDStreamLike, ?, JavaRDD>> dstream =
             (JavaDStreamLike, ?, JavaRDD>>)
             sec.getStream(transform);
-        //noinspection unchecked
         sec.setStream(transform, dstream.mapPartitions(dofn));
       }
     };
@@ -352,21 +347,17 @@ public void evaluate(Window.Bound transform, EvaluationContext context) {
     EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
   }
 
-  private static final Set> UNSUPPORTTED_EVALUATORS = Sets
+  private static final Set> UNSUPPORTED_EVALUATORS = Sets
       .newHashSet();
 
   static {
     //TODO - add support for the following
-    UNSUPPORTTED_EVALUATORS.add(TextIO.Read.Bound.class);
-    UNSUPPORTTED_EVALUATORS.add(TextIO.Write.Bound.class);
-    UNSUPPORTTED_EVALUATORS.add(AvroIO.Read.Bound.class);
-    UNSUPPORTTED_EVALUATORS.add(AvroIO.Write.Bound.class);
-    UNSUPPORTTED_EVALUATORS.add(HadoopIO.Read.Bound.class);
-    UNSUPPORTTED_EVALUATORS.add(HadoopIO.Write.Bound.class);
-  }
-
-  private static > boolean hasTransformEvaluator(Class clazz) {
-    return EVALUATORS.containsKey(clazz);
+    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")
@@ -374,16 +365,16 @@ public void evaluate(Window.Bound transform, EvaluationContext context) {
       getTransformEvaluator(Class clazz, SparkPipelineTranslator rddTranslator) {
     TransformEvaluator transform = (TransformEvaluator) EVALUATORS.get(clazz);
     if (transform == null) {
-      if (UNSUPPORTTED_EVALUATORS.contains(clazz)) {
+      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 PTranform's Input and Output class
-      Class pTOutputClazz = getPTransformOutputClazz(clazz);
-      if (pTOutputClazz == PDone.class) {
+      // 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);
@@ -392,8 +383,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) {
     return transform;
   }
 
-  private static > Class
-      getPTransformOutputClazz(Class clazz) {
+  private static > Class getPTransformOutputClazz(Class clazz) {
     Type[] types = ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments();
     return TypeToken.of(clazz).resolveType(types[1]).getRawType();
   }
@@ -413,12 +403,11 @@ public Translator(SparkPipelineTranslator rddTranslator) {
     @Override
     public boolean hasTranslation(Class> clazz) {
       // streaming includes rdd transformations as well
-      return hasTransformEvaluator(clazz) || rddTranslator.hasTranslation(clazz);
+      return EVALUATORS.containsKey(clazz) || rddTranslator.hasTranslation(clazz);
     }
 
     @Override
-    public TransformEvaluator> translate(
-        Class> clazz) {
+    public > TransformEvaluator translate(Class clazz) {
       return getTransformEvaluator(clazz, rddTranslator);
     }
   }
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
index f9b2d2b75af8..406dfcc78924 100644
--- 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
@@ -47,10 +47,11 @@ public StreamingWindowPipelineDetector(SparkPipelineTranslator translator) {
     super(translator);
   }
 
-  static final TransformTranslator.FieldGetter WINDOW_FG =
+  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 > void
       doVisitTransform(TransformTreeNode node) {
     @SuppressWarnings("unchecked")
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
index 51ba905c4dac..667e949d5392 100644
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java
+++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/CombineGloballyTest.java
@@ -21,7 +21,6 @@
 import com.google.cloud.dataflow.sdk.transforms.Create;
 import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.collect.Iterables;
-import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
 import org.junit.Test;
@@ -47,8 +46,7 @@ public void test() throws Exception {
     res.close();
   }
 
-  public static class WordMerger extends Combine.CombineFn implements
-      Serializable {
+  public static class WordMerger extends Combine.CombineFn {
 
     @Override
     public StringBuilder createAccumulator() {
@@ -78,7 +76,7 @@ public String extractOutput(StringBuilder accumulator) {
     }
 
     private static StringBuilder combine(StringBuilder accum, String datum) {
-      if (null == accum) {
+      if (accum == null) {
         return new StringBuilder(datum);
       } else {
         accum.append(",").append(datum);
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
index b35101873187..579ada554baa 100644
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java
+++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/HadoopFileFormatPipelineTest.java
@@ -24,7 +24,6 @@
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.Reader;
@@ -78,35 +77,28 @@ public void testSequenceFile() throws Exception {
 
     IntWritable key = new IntWritable();
     Text value = new Text();
-    Reader reader = null;
-    try {
-      reader = new Reader(new Configuration(), Reader.file(new Path(outputFile.toURI())));
+    try (Reader reader = new Reader(new Configuration(), Reader.file(new Path(outputFile.toURI())))) {
       int i = 0;
-      while(reader.next(key, value)) {
+      while (reader.next(key, value)) {
         assertEquals(i, key.get());
         assertEquals("value-" + i, value.toString());
         i++;
       }
-    } finally {
-      IOUtils.closeStream(reader);
     }
   }
 
   private void populateFile() throws IOException {
     IntWritable key = new IntWritable();
     Text value = new Text();
-    Writer writer = null;
-    try {
-      writer = SequenceFile.createWriter(new Configuration(),
-          Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class),
-          Writer.file(new Path(this.inputFile.toURI())));
+    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);
       }
-    } finally {
-      IOUtils.closeStream(writer);
     }
   }
 
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
index a8edb3a068b4..21a839b59e6a 100644
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java
+++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SerializationTest.java
@@ -38,6 +38,8 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
+import java.util.regex.Pattern;
+
 import org.junit.Test;
 
 public class SerializationTest {
@@ -126,13 +128,14 @@ public void testRun() throws Exception {
    * A DoFn that tokenizes lines of text into individual words.
    */
   static class ExtractWordsFn extends DoFn {
+    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
     private final Aggregator emptyLines =
         createAggregator("emptyLines", new Sum.SumLongFn());
 
     @Override
     public void processElement(ProcessContext c) {
       // Split the line into words.
-      String[] words = c.element().toString().split("[^a-zA-Z']+");
+      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.
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
index 3d85f467e7f1..1c2f7a9e79a7 100644
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java
+++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/SimpleWordCountTest.java
@@ -31,6 +31,8 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
+import java.util.regex.Pattern;
+
 import org.junit.Test;
 
 public class SimpleWordCountTest {
@@ -60,13 +62,14 @@ public void testRun() throws Exception {
    * A DoFn that tokenizes lines of text into individual words.
    */
   static class ExtractWordsFn extends DoFn {
+    private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+");
     private final Aggregator emptyLines =
         createAggregator("emptyLines", new Sum.SumLongFn());
 
     @Override
     public void processElement(ProcessContext c) {
       // Split the line into words.
-      String[] words = c.element().split("[^a-zA-Z']+");
+      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.
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
index 540bdd9d1a3b..73b36438e538 100644
--- a/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/com/cloudera/dataflow/spark/TransformTranslatorTest.java
@@ -84,7 +84,7 @@ public void testTextIOReadAndWriteTransforms() throws IOException {
     Assert.assertArrayEquals(directOutput.toArray(), sparkOutput.toArray());
   }
 
-  private String runPipeline(String name, PipelineRunner runner) {
+  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 lines =  p.apply(TextIO.Read.from("src/test/resources/test_text.txt"));
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
index d818e9a5146f..087283467f53 100644
--- 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
@@ -27,8 +27,6 @@
 import com.cloudera.dataflow.io.CreateStream;
 import com.cloudera.dataflow.spark.EvaluationResult;
 import com.cloudera.dataflow.spark.SparkPipelineRunner;
-import com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptions;
-import com.cloudera.dataflow.spark.streaming.SparkStreamingPipelineOptionsFactory;
 import com.cloudera.dataflow.spark.streaming.utils.DataflowAssertStreaming;
 
 import org.joda.time.Duration;
@@ -53,7 +51,7 @@ public class FlattenStreamingTest {
           Collections.>singletonList(Arrays.asList(WORDS_ARRAY_2));
   private static final String[] EXPECTED_UNION = {
           "one", "two", "three", "four", "five", "six", "seven", "eight"};
-  final static long TEST_TIMEOUT_MSEC = 1000L;
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
 
   @Test
   public void testRun() throws Exception {
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
index 8778e004ad5c..f68aea88aa43 100644
--- 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
@@ -65,10 +65,10 @@ public class KafkaStreamingTest {
   private static final Set EXPECTED = ImmutableSet.of(
           "k1,v1", "k2,v2", "k3,v3", "k4,v4"
   );
-  private final static long TEST_TIMEOUT_MSEC = 1000L;
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
 
   @BeforeClass
-  public static void init() throws IOException, InterruptedException {
+  public static void init() throws IOException {
     EMBEDDED_ZOOKEEPER.startup();
     EMBEDDED_KAFKA_CLUSTER.startup();
 
@@ -78,12 +78,12 @@ public static void init() throws IOException, InterruptedException {
     producerProps.put("request.required.acks", 1);
     producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
     Serializer stringSerializer = new StringSerializer();
-    @SuppressWarnings("unchecked") KafkaProducer kafkaProducer =
-            new KafkaProducer(producerProps, stringSerializer, stringSerializer);
-    for (Map.Entry en : KAFKA_MESSAGES.entrySet()) {
-      kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
+    try (@SuppressWarnings("unchecked") KafkaProducer kafkaProducer =
+            new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
+      for (Map.Entry en : KAFKA_MESSAGES.entrySet()) {
+        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
+      }
     }
-    kafkaProducer.close();
   }
 
   @Test
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
index eb23b5a32d55..e22e616264fc 100644
--- 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
@@ -45,7 +45,7 @@ public class SimpleStreamingWordCountTest {
       Collections.>singletonList(Arrays.asList(WORDS_ARRAY));
   private static final Set EXPECTED_COUNT_SET =
       ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
-  final static long TEST_TIMEOUT_MSEC = 1000L;
+  private static final long TEST_TIMEOUT_MSEC = 1000L;
 
   @Test
   public void testRun() throws Exception {
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
index 6daae54c0f60..e75d7295c212 100644
--- 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
@@ -32,11 +32,16 @@
 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 ports;
   private final String zkConnection;
   private final Properties baseProperties;
@@ -59,28 +64,28 @@ public EmbeddedKafkaCluster(String zkConnection, Properties baseProperties, List
     this.ports = resolvePorts(ports);
     this.baseProperties = baseProperties;
 
-    this.brokers = new ArrayList();
-    this.logDirs = new ArrayList();
+    this.brokers = new ArrayList<>();
+    this.logDirs = new ArrayList<>();
 
     this.brokerList = constructBrokerList(this.ports);
   }
 
-  private List resolvePorts(List ports) {
-    List resolvedPorts = new ArrayList();
+  private static List resolvePorts(List ports) {
+    List resolvedPorts = new ArrayList<>();
     for (Integer port : ports) {
       resolvedPorts.add(resolvePort(port));
     }
     return resolvedPorts;
   }
 
-  private int resolvePort(int port) {
+  private static int resolvePort(int port) {
     if (port == -1) {
       return TestUtils.getAvailablePort();
     }
     return port;
   }
 
-  private String constructBrokerList(List ports) {
+  private static String constructBrokerList(List ports) {
     StringBuilder sb = new StringBuilder();
     for (Integer port : ports) {
       if (sb.length() > 0) {
@@ -113,7 +118,7 @@ public void startup() {
   }
 
 
-  private KafkaServer startBroker(Properties props) {
+  private static KafkaServer startBroker(Properties props) {
     KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
     server.startup();
     return server;
@@ -144,24 +149,21 @@ public void shutdown() {
       try {
         broker.shutdown();
       } catch (Exception e) {
-        e.printStackTrace();
+        LOG.warn("{}", e.getMessage(), e);
       }
     }
     for (File logDir : logDirs) {
       try {
         TestUtils.deleteFile(logDir);
       } catch (FileNotFoundException e) {
-        e.printStackTrace();
+        LOG.warn("{}", e.getMessage(), e);
       }
     }
   }
 
   @Override
   public String toString() {
-    final StringBuilder sb = new StringBuilder("EmbeddedKafkaCluster{");
-    sb.append("brokerList='").append(brokerList).append('\'');
-    sb.append('}');
-    return sb.toString();
+    return "EmbeddedKafkaCluster{" + "brokerList='" + brokerList + "'}";
   }
 
   public static class EmbeddedZookeeper {
@@ -185,7 +187,7 @@ public EmbeddedZookeeper(int port, int tickTime) {
       this.tickTime = tickTime;
     }
 
-    private int resolvePort(int port) {
+    private static int resolvePort(int port) {
       if (port == -1) {
         return TestUtils.getAvailablePort();
       }
@@ -198,8 +200,8 @@ public void startup() throws IOException {
       }
       this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
               1024);
-      this.snapshotDir = TestUtils.constructTempDir("embeeded-zk/snapshot");
-      this.logDir = TestUtils.constructTempDir("embeeded-zk/log");
+      this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
+      this.logDir = TestUtils.constructTempDir("embedded-zk/log");
 
       try {
         factory.startup(new ZooKeeperServer(snapshotDir, logDir, tickTime));
@@ -245,22 +247,22 @@ public int getTickTime() {
 
     @Override
     public String toString() {
-      final StringBuilder sb = new StringBuilder("EmbeddedZookeeper{");
-      sb.append("connection=").append(getConnection());
-      sb.append('}');
-      return sb.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);
@@ -270,13 +272,13 @@ public void sleep(long ms) {
     }
   }
 
-  static class TestUtils {
+  static final class TestUtils {
     private static final Random RANDOM = new Random();
 
     private TestUtils() {
     }
 
-    public static File constructTempDir(String dirPrefix) {
+    static File constructTempDir(String dirPrefix) {
       File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt
               (10000000));
       if (!file.mkdirs()) {
@@ -286,20 +288,17 @@ public static File constructTempDir(String dirPrefix) {
       return file;
     }
 
-    public static int getAvailablePort() {
+    static int getAvailablePort() {
       try {
-        ServerSocket socket = new ServerSocket(0);
-        try {
+        try (ServerSocket socket = new ServerSocket(0)) {
           return socket.getLocalPort();
-        } finally {
-          socket.close();
         }
       } catch (IOException e) {
         throw new IllegalStateException("Cannot find available port: " + e.getMessage(), e);
       }
     }
 
-    public static boolean deleteFile(File path) throws FileNotFoundException {
+    static boolean deleteFile(File path) throws FileNotFoundException {
       if (!path.exists()) {
         throw new FileNotFoundException(path.getAbsolutePath());
       }

From 1229b009aab111734ac9dc488cc50099285a43f6 Mon Sep 17 00:00:00 2001
From: Sean Owen 
Date: Fri, 22 Jan 2016 15:20:48 +0000
Subject: [PATCH 137/137] First wave of changes from feedback

---
 .../java/com/cloudera/dataflow/spark/EvaluationContext.java | 2 --
 .../com/cloudera/dataflow/spark/SparkPipelineRunner.java    | 2 +-
 .../com/cloudera/dataflow/spark/TransformTranslator.java    | 6 +-----
 3 files changed, 2 insertions(+), 8 deletions(-)

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
index 356acabbc1ee..a6ac6c2f3e86 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/EvaluationContext.java
@@ -49,7 +49,6 @@ public class EvaluationContext implements EvaluationResult {
   private final JavaSparkContext jsc;
   private final Pipeline pipeline;
   private final SparkRuntimeContext runtime;
-  //private final CoderRegistry registry;
   private final Map> pcollections = new LinkedHashMap<>();
   private final Set> leafRdds = new LinkedHashSet<>();
   private final Set multireads = new LinkedHashSet<>();
@@ -60,7 +59,6 @@ public class EvaluationContext implements EvaluationResult {
   public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
     this.jsc = jsc;
     this.pipeline = pipeline;
-    //this.registry = pipeline.getCoderRegistry();
     this.runtime = new SparkRuntimeContext(jsc, pipeline);
   }
 
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
index 5287f2011732..a9c2d860b8fe 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/SparkPipelineRunner.java
@@ -213,7 +213,7 @@ public void enterCompositeTransform(TransformTreeNode node) {
             (Class>) node.getTransform().getClass();
         if (translator.hasTranslation(transformClass)) {
           LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName());
-          LOG.debug("Composite transform class: '{}'", node.getTransform().getClass());
+          LOG.debug("Composite transform class: '{}'", transformClass);
           currentTranslatedCompositeNode = node;
         }
       }
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
index db335edbd89a..58b1924d555c 100644
--- a/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
+++ b/runners/spark/src/main/java/com/cloudera/dataflow/spark/TransformTranslator.java
@@ -772,10 +772,6 @@ private static Map, BroadcastHelper> getSideInputs(
     EVALUATORS.put(Window.Bound.class, window());
   }
 
-  public static > boolean hasTransformEvaluator(Class clazz) {
-    return EVALUATORS.containsKey(clazz);
-  }
-
   public static > TransformEvaluator
   getTransformEvaluator(Class clazz) {
     @SuppressWarnings("unchecked")
@@ -793,7 +789,7 @@ public static class Translator implements SparkPipelineTranslator {
 
     @Override
     public boolean hasTranslation(Class> clazz) {
-      return hasTransformEvaluator(clazz);
+      return EVALUATORS.containsKey(clazz);
     }
 
     @Override