From 767e3ae1efdf0d821ff05e0cda38159e778493c9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Feb 2014 10:59:02 -0800 Subject: [PATCH 01/32] Fix typos in Spark Streaming programming guide Author: Andrew Or Closes #536 from andrewor14/streaming-typos and squashes the following commits: a05faa6 [Andrew Or] Fix broken link and wording bc2e4bc [Andrew Or] Merge github.com:apache/incubator-spark into streaming-typos d5515b4 [Andrew Or] TD's comments 767ef12 [Andrew Or] Fix broken links 8f4c731 [Andrew Or] Fix typos in programming guide --- docs/streaming-programming-guide.md | 27 +++++++++++++-------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6b3262fbd7831..924f0f4306bc2 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -428,9 +428,9 @@ KafkaUtils.createStream(javaStreamingContext, kafkaParams, ...); -For more details on these additional sources, see the corresponding [API documentation] -(#where-to-go-from-here). Furthermore, you can also implement your own custom receiver -for your sources. See the [Custom Receiver Guide](streaming-custom-receivers.html). +For more details on these additional sources, see the corresponding [API documentation](#where-to-go-from-here). +Furthermore, you can also implement your own custom receiver for your sources. See the +[Custom Receiver Guide](streaming-custom-receivers.html). ## Operations There are two kinds of DStream operations - _transformations_ and _output operations_. Similar to @@ -520,9 +520,8 @@ The last two transformations are worth highlighting again.

UpdateStateByKey Operation

-The `updateStateByKey` operation allows -you to main arbitrary stateful computation, where you want to maintain some state data and -continuously update it with new information. To use this, you will have to do two steps. +The `updateStateByKey` operation allows you to maintain arbitrary state while continuously updating +it with new information. To use this, you will have to do two steps. 1. Define the state - The state can be of arbitrary data type. 1. Define the state update function - Specify with a function how to update the state using the @@ -925,7 +924,7 @@ exception saying so. ## Monitoring Besides Spark's in-built [monitoring capabilities](monitoring.html), the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface, +(api/streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface, which allows you to get statistics of batch processing times, queueing delays, and total end-to-end delays. Note that this is still an experimental API and it is likely to be improved upon (i.e., more information reported) in the future. @@ -1000,11 +999,11 @@ Since all data is modeled as RDDs with their lineage of deterministic operations for output operations. ## Failure of the Driver Node -To allows a streaming application to operate 24/7, Spark Streaming allows a streaming computation +For a streaming application to operate 24/7, Spark Streaming allows a streaming computation to be resumed even after the failure of the driver node. Spark Streaming periodically writes the metadata information of the DStreams setup through the `StreamingContext` to a HDFS directory (can be any Hadoop-compatible filesystem). This periodic -*checkpointing* can be enabled by setting a the checkpoint +*checkpointing* can be enabled by setting the checkpoint directory using `ssc.checkpoint()` as described [earlier](#rdd-checkpointing). On failure of the driver node, the lost `StreamingContext` can be recovered from this information, and restarted. @@ -1105,8 +1104,8 @@ classes. So, if you are using `getOrCreate`, then make sure that the checkpoint explicitly deleted every time recompiled code needs to be launched. This failure recovery can be done automatically using Spark's -[standalone cluster mode](spark-standalone.html), which allows any Spark -application's driver to be as well as, ensures automatic restart of the driver on failure (see +[standalone cluster mode](spark-standalone.html), which allows the driver of any Spark application +to be launched within the cluster and be restarted on failure (see [supervise mode](spark-standalone.html#launching-applications-inside-the-cluster)). This can be tested locally by launching the above example using the supervise mode in a local standalone cluster and killing the java process running the driver (will be shown as @@ -1123,7 +1122,7 @@ There are two different failure behaviors based on which input sources are used. 1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure. 1. _Using any input source that receives data through a network_ - The received input data is -replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost +replicated in memory to multiple nodes. Since all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state @@ -1133,11 +1132,11 @@ In future releases, we will support full recoverability for all input sources. N non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data. -To better understand the behavior of the system under driver failure with a HDFS source, lets +To better understand the behavior of the system under driver failure with a HDFS source, let's consider what will happen with a file input stream. Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further -in the case of file input stream, we shall use an example. Lets say, files are being generated +in the case of file input stream, we shall use an example. Let's say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure. From e0d49ad229e2047bd6dda8e66341aff8f2a122a2 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 17 Feb 2014 15:12:52 -0800 Subject: [PATCH 02/32] [SPARK-1090] improvement on spark_shell (help information, configure memory) https://spark-project.atlassian.net/browse/SPARK-1090 spark-shell should print help information about parameters and should allow user to configure exe memory there is no document about hot to set --cores/-c in spark-shell and also users should be able to set executor memory through command line options In this PR I also check the format of the options passed by the user Author: CodingCat Closes #599 from CodingCat/spark_shell_improve and squashes the following commits: de5aa38 [CodingCat] add parameter to set driver memory 915cbf8 [CodingCat] improvement on spark_shell (help information, configure memory) --- bin/spark-shell | 48 ++++++++++++++++--- .../org/apache/spark/repl/SparkILoop.scala | 2 +- 2 files changed, 43 insertions(+), 7 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 05a46ee0caf55..2bff06cf70051 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -21,8 +21,6 @@ # Shell script for starting the Spark Shell REPL # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} # if those two env vars are set in spark-env.sh but MASTER is not. -# Options: -# -c Set the number of cores for REPL to use cygwin=false case "`uname`" in @@ -32,14 +30,52 @@ esac # Enter posix mode for bash set -o posix +CORE_PATTERN="^[0-9]+$" +MEM_PATTERN="^[0-9]+[m|g|M|G]$" + FWDIR="$(cd `dirname $0`/..; pwd)" +if [ "$1" = "--help" ] || [ "$1" = "-h" ]; then + echo "Usage: spark-shell [OPTIONS]" + echo "OPTIONS:" + echo "-c --cores num, the maximum number of cores to be used by the spark shell" + echo "-em --execmem num[m|g], the memory used by each executor of spark shell" + echo "-dm --drivermem num[m|g], the memory used by the spark shell and driver" + echo "-h --help, print this help information" + exit +fi + +SPARK_SHELL_OPTS="" + for o in "$@"; do if [ "$1" = "-c" -o "$1" = "--cores" ]; then shift - if [ -n "$1" ]; then - OPTIONS="-Dspark.cores.max=$1" + if [[ "$1" =~ $CORE_PATTERN ]]; then + SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.cores.max=$1" shift + else + echo "ERROR: wrong format for -c/--cores" + exit 1 + fi + fi + if [ "$1" = "-em" -o "$1" = "--execmem" ]; then + shift + if [[ $1 =~ $MEM_PATTERN ]]; then + SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.executor.memory=$1" + shift + else + echo "ERROR: wrong format for --execmem/-em" + exit 1 + fi + fi + if [ "$1" = "-dm" -o "$1" = "--drivermem" ]; then + shift + if [[ $1 =~ $MEM_PATTERN ]]; then + export SPARK_MEM=$1 + shift + else + echo "ERROR: wrong format for --drivermem/-dm" + exit 1 fi fi done @@ -95,10 +131,10 @@ if $cygwin; then # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class -Djline.terminal=unix $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@" stty icanon echo > /dev/null 2>&1 else - $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@" fi # record the exit status lest it be overwritten: diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index bc25b50a4efc7..013cea07d48fd 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -954,7 +954,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, conf.setSparkHome(System.getenv("SPARK_HOME")) } sparkContext = new SparkContext(conf) - echo("Created spark context..") + logInfo("Created spark context..") sparkContext } From f74ae0ebcee59b70a56d34bdf63e3d1b38e2bd59 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 17 Feb 2014 19:23:27 -0800 Subject: [PATCH 03/32] SPARK-1098: Minor cleanup of ClassTag usage in Java API Our usage of fake ClassTags in this manner is probably not healthy, but I'm not sure if there's a better solution available, so I just cleaned up and documented the current one. Author: Aaron Davidson Closes #604 from aarondav/master and squashes the following commits: b398e89 [Aaron Davidson] SPARK-1098: Minor cleanup of ClassTag usage in Java API --- .../apache/spark/api/java/JavaPairRDD.scala | 82 +++++++++--------- .../org/apache/spark/api/java/JavaRDD.scala | 4 +- .../apache/spark/api/java/JavaRDDLike.scala | 37 ++++---- .../spark/api/java/JavaSparkContext.scala | 85 +++++++++++-------- 4 files changed, 108 insertions(+), 100 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index cd0aea0cb3d1f..3f672900cb90f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -17,35 +17,29 @@ package org.apache.spark.api.java -import java.util.{List => JList} -import java.util.Comparator +import java.util.{Comparator, List => JList} -import scala.Tuple2 import scala.collection.JavaConversions._ import scala.reflect.ClassTag import com.google.common.base.Optional +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputFormat +import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.conf.Configuration -import org.apache.spark.HashPartitioner -import org.apache.spark.Partitioner +import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions -import org.apache.spark.api.java.function.{Function2 => JFunction2} -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.partial.BoundedDouble -import org.apache.spark.partial.PartialResult -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.OrderedRDDFunctions +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel - -class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K], - implicit val vClassTag: ClassTag[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { +class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) + (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) + extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) @@ -158,7 +152,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val ctag: ClassTag[C] = fakeClassTag fromRDD(rdd.combineByKey( createCombiner, mergeValue, @@ -284,19 +278,19 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * RDD will be <= us. */ def subtractByKey[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, V] = { - implicit val cmw: ClassTag[W] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val ctag: ClassTag[W] = fakeClassTag fromRDD(rdd.subtractByKey(other)) } /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ def subtractByKey[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, V] = { - implicit val cmw: ClassTag[W] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val ctag: ClassTag[W] = fakeClassTag fromRDD(rdd.subtractByKey(other, numPartitions)) } /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ def subtractByKey[W](other: JavaPairRDD[K, W], p: Partitioner): JavaPairRDD[K, V] = { - implicit val cmw: ClassTag[W] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val ctag: ClassTag[W] = fakeClassTag fromRDD(rdd.subtractByKey(other, p)) } @@ -345,7 +339,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val ctag: ClassTag[C] = fakeClassTag fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd))) } @@ -438,7 +432,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * this also retains the original RDD's partitioning. */ def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { - implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.mapValues(f)) } @@ -449,7 +443,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } @@ -682,31 +676,35 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K } object JavaPairRDD { - def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassTag[K], - vcm: ClassTag[T]): RDD[(K, JList[T])] = - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _) - - def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K], - vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd) - .mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) - - def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1], - Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1], - JList[W2]))] = rddToPairRDDFunctions(rdd).mapValues( - (x: (Seq[V], Seq[W1], Seq[W2])) => (seqAsJavaList(x._1), - seqAsJavaList(x._2), - seqAsJavaList(x._3))) - - def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = + private[spark] + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { + rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + } + + private[spark] + def cogroupResultToJava[K: ClassTag, V, W]( + rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + } + + private[spark] + def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( + rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rddToPairRDDFunctions(rdd) + .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + } + + def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { new JavaPairRDD[K, V](rdd) + } implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd /** Convert a JavaRDD of key-value pairs to JavaPairRDD. */ def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = { - implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val cmv: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val ctagK: ClassTag[K] = fakeClassTag + implicit val ctagV: ClassTag[V] = fakeClassTag new JavaPairRDD[K, V](rdd.rdd) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 7d48ce01cf2cc..7b73057953552 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -24,8 +24,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel -class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends -JavaRDDLike[T, JavaRDD[T]] { +class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) + extends JavaRDDLike[T, JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index fcb9729c10a6d..24a9925dbd22c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -17,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{List => JList, Comparator} +import java.util.{Comparator, List => JList} + import scala.Tuple2 import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -25,14 +26,14 @@ import scala.reflect.ClassTag import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.spark.{SparkContext, Partition, TaskContext} -import org.apache.spark.rdd.RDD +import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.api.java.JavaPairRDD._ -import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} -import org.apache.spark.partial.{PartialResult, BoundedDouble} +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel - trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -88,8 +89,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to all elements of this RDD. */ def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - def cm = implicitly[ClassTag[Tuple2[_, _]]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] - new JavaPairRDD(rdd.map(f)(cm))(f.keyType(), f.valueType()) + val ctag = implicitly[ClassTag[Tuple2[K2, V2]]] + new JavaPairRDD(rdd.map(f)(ctag))(f.keyType(), f.valueType()) } /** @@ -119,8 +120,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassTag[Tuple2[_, _]]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] - JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType()) + val ctag = implicitly[ClassTag[Tuple2[K2, V2]]] + JavaPairRDD.fromRDD(rdd.flatMap(fn)(ctag))(f.keyType(), f.valueType()) } /** @@ -202,10 +203,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val vcm: ClassTag[JList[T]] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm) + implicit val ctagK: ClassTag[K] = fakeClassTag + implicit val ctagV: ClassTag[JList[T]] = fakeClassTag + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType))) } /** @@ -213,10 +213,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val vcm: ClassTag[JList[T]] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) + implicit val ctagK: ClassTag[K] = fakeClassTag + implicit val ctagV: ClassTag[JList[T]] = fakeClassTag + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType))) } /** @@ -407,7 +406,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Creates tuples of the elements in this RDD by applying `f`. */ def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = { - implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val ctag: ClassTag[K] = fakeClassTag JavaPairRDD.fromRDD(rdd.keyBy(f)) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 22dc9c9e2ecfe..dc26b7f621fee 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -23,19 +23,17 @@ import scala.collection.JavaConversions import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import com.google.common.base.Optional import org.apache.spark._ -import org.apache.spark.SparkContext.IntAccumulatorParam -import org.apache.spark.SparkContext.DoubleAccumulatorParam +import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD - /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. @@ -96,7 +94,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val ctag: ClassTag[T] = fakeClassTag sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } @@ -107,8 +105,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int) : JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val vcm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val ctagK: ClassTag[K] = fakeClassTag + implicit val ctagV: ClassTag[V] = fakeClassTag JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) } @@ -149,8 +147,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits)) } @@ -163,8 +161,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass)) } @@ -176,8 +174,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - sc.objectFile(path, minSplits)(cm) + implicit val ctag: ClassTag[T] = fakeClassTag + sc.objectFile(path, minSplits)(ctag) } /** @@ -188,8 +186,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - sc.objectFile(path)(cm) + implicit val ctag: ClassTag[T] = fakeClassTag + sc.objectFile(path)(ctag) } /** @@ -209,8 +207,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits)) } @@ -229,8 +227,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork keyClass: Class[K], valueClass: Class[V] ): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass)) } @@ -248,8 +246,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)) } @@ -266,8 +264,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork keyClass: Class[K], valueClass: Class[V] ): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(keyClass) - implicit val vcm: ClassTag[V] = ClassTag(valueClass) + implicit val ctagK: ClassTag[K] = ClassTag(keyClass) + implicit val ctagV: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass)) } @@ -287,8 +285,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork kClass: Class[K], vClass: Class[V], conf: Configuration): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(kClass) - implicit val vcm: ClassTag[V] = ClassTag(vClass) + implicit val ctagK: ClassTag[K] = ClassTag(kClass) + implicit val ctagV: ClassTag[V] = ClassTag(vClass) new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf)) } @@ -306,26 +304,26 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork fClass: Class[F], kClass: Class[K], vClass: Class[V]): JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = ClassTag(kClass) - implicit val vcm: ClassTag[V] = ClassTag(vClass) + implicit val ctagK: ClassTag[K] = ClassTag(kClass) + implicit val ctagV: ClassTag[V] = ClassTag(vClass) new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass)) } /** Build the union of two or more RDDs. */ override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = { val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) - implicit val cm: ClassTag[T] = first.classTag - sc.union(rdds)(cm) + implicit val ctag: ClassTag[T] = first.classTag + sc.union(rdds) } /** Build the union of two or more RDDs. */ override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]]) : JavaPairRDD[K, V] = { val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) - implicit val cm: ClassTag[(K, V)] = first.classTag - implicit val kcm: ClassTag[K] = first.kClassTag - implicit val vcm: ClassTag[V] = first.vClassTag - new JavaPairRDD(sc.union(rdds)(cm))(kcm, vcm) + implicit val ctag: ClassTag[(K, V)] = first.classTag + implicit val ctagK: ClassTag[K] = first.kClassTag + implicit val ctagV: ClassTag[V] = first.vClassTag + new JavaPairRDD(sc.union(rdds)) } /** Build the union of two or more RDDs. */ @@ -447,8 +445,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def getCheckpointDir = JavaUtils.optionToOptional(sc.getCheckpointDir) protected def checkpointFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val ctag: ClassTag[T] = fakeClassTag new JavaRDD(sc.checkpointFile(path)) } @@ -535,4 +532,18 @@ object JavaSparkContext { * your driver program. */ def jarOfObject(obj: AnyRef): Array[String] = SparkContext.jarOfObject(obj).toArray + + /** + * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef]. + * + * This method is used to keep ClassTags out of the external Java API, as the Java compiler + * cannot produce them automatically. While this ClassTag-faking does please the compiler, + * it can cause problems at runtime if the Scala API relies on ClassTags for correctness. + * + * Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior, just worse performance + * or security issues. For instance, an Array[AnyRef] can hold any type T, but may lose primitive + * specialization. + */ + private[spark] + def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] } From ccb327a49a7323efd98a33223c438a670bba7cec Mon Sep 17 00:00:00 2001 From: NirmalReddy Date: Tue, 18 Feb 2014 14:44:36 -0800 Subject: [PATCH 04/32] Optimized imports Optimized imports and arranged according to scala style guide @ https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Imports Author: NirmalReddy Author: NirmalReddy Closes #613 from NirmalReddy/opt-imports and squashes the following commits: 578b4f5 [NirmalReddy] imported java.lang.Double as JDouble a2cbcc5 [NirmalReddy] addressed the comments 776d664 [NirmalReddy] Optimized imports in core --- .../spark/network/netty/FileClient.java | 4 +- .../network/netty/FileServerHandler.java | 4 +- .../mapreduce/SparkHadoopMapReduceUtil.scala | 3 +- .../scala/org/apache/spark/Accumulators.scala | 3 +- .../spark/BlockStoreShuffleFetcher.scala | 3 +- .../scala/org/apache/spark/CacheManager.scala | 4 +- .../scala/org/apache/spark/FutureAction.scala | 4 +- .../org/apache/spark/HttpFileServer.scala | 4 +- .../scala/org/apache/spark/HttpServer.scala | 2 +- .../org/apache/spark/MapOutputTracker.scala | 1 - .../apache/spark/SerializableWritable.scala | 2 +- .../org/apache/spark/ShuffleFetcher.scala | 2 - .../scala/org/apache/spark/SparkConf.scala | 2 - .../scala/org/apache/spark/SparkContext.scala | 15 +++---- .../scala/org/apache/spark/SparkEnv.scala | 9 ++-- .../org/apache/spark/SparkHadoopWriter.scala | 2 +- .../apache/spark/api/java/JavaDoubleRDD.scala | 44 +++++++++---------- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../java/function/DoubleFlatMapFunction.java | 1 - .../api/java/function/DoubleFunction.java | 1 - .../spark/api/java/function/Function.java | 5 +-- .../spark/api/java/function/Function2.java | 4 +- .../spark/api/java/function/Function3.java | 5 +-- .../java/function/PairFlatMapFunction.java | 4 +- .../spark/api/java/function/PairFunction.java | 4 +- .../spark/api/python/PythonPartitioner.scala | 2 - .../apache/spark/api/python/PythonRDD.scala | 2 +- .../api/python/PythonWorkerFactory.scala | 4 +- .../spark/broadcast/HttpBroadcast.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 1 - .../org/apache/spark/deploy/Client.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 4 +- .../apache/spark/deploy/JsonProtocol.scala | 3 +- .../spark/deploy/LocalSparkCluster.scala | 6 +-- .../spark/deploy/client/TestClient.scala | 6 +-- .../spark/deploy/master/ApplicationInfo.scala | 7 ++- .../master/FileSystemPersistenceEngine.scala | 1 + .../apache/spark/deploy/master/Master.scala | 5 +-- .../spark/deploy/master/MasterArguments.scala | 2 +- .../deploy/master/SparkZooKeeperSession.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 2 +- .../master/ZooKeeperPersistenceEngine.scala | 4 +- .../deploy/master/ui/ApplicationPage.scala | 3 +- .../spark/deploy/master/ui/IndexPage.scala | 4 +- .../spark/deploy/master/ui/MasterWebUI.scala | 1 + .../spark/deploy/worker/CommandUtils.scala | 2 +- .../spark/deploy/worker/ExecutorRunner.scala | 3 +- .../apache/spark/deploy/worker/Worker.scala | 1 - .../spark/deploy/worker/WorkerArguments.scala | 3 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 4 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../spark/executor/ExecutorBackend.scala | 1 + .../spark/executor/ExecutorSource.scala | 5 +-- .../spark/executor/MesosExecutorBackend.scala | 4 +- .../apache/spark/io/CompressionCodec.scala | 3 +- .../apache/spark/metrics/MetricsConfig.scala | 2 +- .../apache/spark/metrics/MetricsSystem.scala | 6 +-- .../spark/metrics/sink/ConsoleSink.scala | 4 +- .../apache/spark/metrics/sink/CsvSink.scala | 4 +- .../spark/metrics/sink/GangliaSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 4 +- .../apache/spark/metrics/sink/JmxSink.scala | 4 +- .../spark/metrics/sink/MetricsServlet.scala | 8 ++-- .../apache/spark/network/BufferMessage.scala | 1 - .../org/apache/spark/network/Connection.scala | 11 ++--- .../spark/network/ConnectionManager.scala | 13 +++--- .../spark/network/ConnectionManagerId.scala | 1 - .../spark/network/ConnectionManagerTest.scala | 9 ++-- .../org/apache/spark/network/Message.scala | 3 +- .../apache/spark/network/MessageChunk.scala | 1 - .../spark/network/MessageChunkHeader.scala | 1 - .../apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../spark/network/netty/FileHeader.scala | 2 +- .../spark/network/netty/ShuffleCopier.scala | 7 ++- .../spark/network/netty/ShuffleSender.scala | 1 - .../partial/ApproximateActionListener.scala | 2 +- .../spark/partial/GroupedCountEvaluator.scala | 4 +- .../spark/partial/GroupedMeanEvaluator.scala | 5 +-- .../spark/partial/GroupedSumEvaluator.scala | 5 +-- .../scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../org/apache/spark/rdd/CartesianRDD.scala | 5 ++- .../org/apache/spark/rdd/CheckpointRDD.scala | 7 ++- .../org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 7 +-- .../apache/spark/rdd/DoubleRDDFunctions.scala | 4 +- .../org/apache/spark/rdd/FilteredRDD.scala | 3 +- .../org/apache/spark/rdd/FlatMappedRDD.scala | 2 +- .../spark/rdd/FlatMappedValuesRDD.scala | 3 +- .../org/apache/spark/rdd/GlommedRDD.scala | 3 +- .../org/apache/spark/rdd/HadoopRDD.scala | 3 +- .../apache/spark/rdd/MapPartitionsRDD.scala | 3 +- .../apache/spark/rdd/MappedValuesRDD.scala | 3 +- .../apache/spark/rdd/PairRDDFunctions.scala | 10 ++--- .../spark/rdd/ParallelCollectionRDD.scala | 7 +-- .../spark/rdd/PartitionPruningRDD.scala | 3 +- .../spark/rdd/PartitionerAwareUnionRDD.scala | 5 ++- .../spark/rdd/PartitionwiseSampledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 5 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 15 +++---- .../apache/spark/rdd/RDDCheckpointData.scala | 3 +- .../org/apache/spark/rdd/SampledRDD.scala | 3 +- .../spark/rdd/SequenceFileRDDFunctions.scala | 8 ++-- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../org/apache/spark/rdd/SubtractedRDD.scala | 9 ++-- .../scala/org/apache/spark/rdd/UnionRDD.scala | 6 +-- .../spark/rdd/ZippedPartitionsRDD.scala | 6 ++- .../org/apache/spark/rdd/ZippedRDD.scala | 6 +-- .../apache/spark/rdd/ZippedWithIndexRDD.scala | 2 +- .../apache/spark/scheduler/ActiveJob.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../spark/scheduler/DAGSchedulerSource.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 14 +++--- .../apache/spark/scheduler/JobLogger.scala | 4 +- .../apache/spark/scheduler/MapStatus.scala | 3 +- .../apache/spark/scheduler/ResultTask.scala | 2 +- .../apache/spark/scheduler/Schedulable.scala | 3 +- .../spark/scheduler/SchedulableBuilder.scala | 4 +- .../spark/scheduler/SchedulerBackend.scala | 2 - .../spark/scheduler/ShuffleMapTask.scala | 5 +-- .../spark/scheduler/SparkListener.scala | 3 +- .../org/apache/spark/scheduler/Task.scala | 1 - .../spark/scheduler/TaskDescription.scala | 1 + .../org/apache/spark/scheduler/TaskInfo.scala | 2 - .../apache/spark/scheduler/TaskLocality.scala | 1 - .../apache/spark/scheduler/TaskResult.scala | 7 +-- .../spark/scheduler/TaskResultGetter.scala | 1 - .../spark/scheduler/TaskSchedulerImpl.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 4 +- .../cluster/CoarseGrainedClusterMessage.scala | 3 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../mesos/CoarseMesosSchedulerBackend.scala | 7 ++- .../cluster/mesos/MesosSchedulerBackend.scala | 7 ++- .../spark/scheduler/local/LocalBackend.scala | 2 +- .../spark/serializer/JavaSerializer.scala | 2 +- .../spark/serializer/KryoSerializer.scala | 8 ++-- .../apache/spark/serializer/Serializer.scala | 3 +- .../spark/serializer/SerializerManager.scala | 2 +- .../spark/storage/BlockFetcherIterator.scala | 5 +-- .../apache/spark/storage/BlockManager.scala | 13 +++--- .../apache/spark/storage/BlockManagerId.scala | 1 + .../spark/storage/BlockManagerMaster.scala | 2 +- .../storage/BlockManagerMasterActor.scala | 2 +- .../spark/storage/BlockManagerMessages.scala | 1 - .../storage/BlockManagerSlaveActor.scala | 1 - .../spark/storage/BlockManagerSource.scala | 3 +- .../spark/storage/BlockManagerWorker.scala | 2 +- .../apache/spark/storage/BlockMessage.scala | 2 +- .../org/apache/spark/storage/BlockStore.scala | 1 + .../org/apache/spark/storage/DiskStore.scala | 1 - .../apache/spark/storage/MemoryStore.scala | 7 +-- .../spark/storage/StoragePerfTester.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 4 +- .../apache/spark/storage/ThreadingTest.scala | 7 +-- .../org/apache/spark/ui/JettyUtils.scala | 10 ++--- .../scala/org/apache/spark/ui/SparkUI.scala | 6 +-- .../apache/spark/ui/UIWorkloadGenerator.scala | 1 - .../apache/spark/ui/env/EnvironmentUI.scala | 5 +-- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 1 - .../spark/ui/jobs/JobProgressListener.scala | 1 - .../apache/spark/ui/jobs/JobProgressUI.scala | 14 ++---- .../org/apache/spark/ui/jobs/PoolPage.scala | 6 +-- .../org/apache/spark/ui/jobs/StagePage.scala | 5 +-- .../org/apache/spark/ui/jobs/StageTable.scala | 3 +- .../spark/ui/storage/BlockManagerUI.scala | 2 - .../apache/spark/ui/storage/IndexPage.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 3 +- .../org/apache/spark/util/AkkaUtils.scala | 2 +- .../spark/util/BoundedPriorityQueue.scala | 3 +- .../spark/util/ByteBufferInputStream.scala | 1 + .../apache/spark/util/ClosureCleaner.scala | 4 +- .../apache/spark/util/MetadataCleaner.scala | 4 +- .../org/apache/spark/util/MutablePair.scala | 1 - .../spark/util/SerializableBuffer.scala | 2 +- .../spark/util/SerializableHyperLogLog.scala | 5 ++- .../org/apache/spark/util/SizeEstimator.scala | 11 +++-- .../spark/util/TimeStampedHashMap.scala | 5 ++- .../spark/util/TimeStampedHashSet.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 12 ++--- .../scala/org/apache/spark/util/Vector.scala | 1 + .../apache/spark/util/collection/BitSet.scala | 1 - .../collection/ExternalAppendOnlyMap.scala | 2 +- .../spark/util/random/RandomSampler.scala | 1 + .../spark/util/random/XORShiftRandom.scala | 1 + .../org/apache/spark/AccumulatorSuite.scala | 7 ++- .../org/apache/spark/CheckpointSuite.scala | 7 ++- .../org/apache/spark/DistributedSuite.scala | 9 ++-- .../scala/org/apache/spark/DriverSuite.scala | 1 + .../scala/org/apache/spark/FailureSuite.scala | 2 +- .../org/apache/spark/FileServerSuite.scala | 3 +- .../scala/org/apache/spark/FileSuite.scala | 9 ++-- .../scala/org/apache/spark/JavaAPISuite.java | 5 +-- .../apache/spark/JobCancellationSuite.scala | 3 +- .../org/apache/spark/LocalSparkContext.scala | 7 ++- .../apache/spark/MapOutputTrackerSuite.scala | 5 ++- .../org/apache/spark/PartitioningSuite.scala | 3 +- .../org/apache/spark/PipedRDDSuite.scala | 1 - .../org/apache/spark/SharedSparkContext.scala | 2 +- .../org/apache/spark/ShuffleNettySuite.scala | 1 - .../scala/org/apache/spark/ShuffleSuite.scala | 5 +-- .../apache/spark/SparkContextInfoSuite.scala | 1 - .../org/apache/spark/ThreadingSuite.scala | 3 -- .../org/apache/spark/UnpersistSuite.scala | 3 +- .../apache/spark/ZippedPartitionsSuite.scala | 9 ---- .../spark/api/python/PythonRDDSuite.scala | 5 +-- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../deploy/worker/DriverRunnerTest.scala | 6 +-- .../deploy/worker/ExecutorRunnerTest.scala | 2 +- .../deploy/worker/WorkerWatcherSuite.scala | 5 +-- .../spark/io/CompressionCodecSuite.scala | 2 +- .../spark/metrics/MetricsSystemSuite.scala | 3 +- .../spark/rdd/AsyncRDDActionsSuite.scala | 1 - .../org/apache/spark/rdd/DoubleRDDSuite.scala | 6 +-- .../org/apache/spark/rdd/JdbcRDDSuite.scala | 7 +-- .../spark/rdd/PairRDDFunctionsSuite.scala | 3 +- .../rdd/ParallelCollectionSplitSuite.scala | 4 +- .../spark/rdd/PartitionPruningRDDSuite.scala | 2 +- .../rdd/PartitionwiseSampledRDDSuite.scala | 1 + .../scala/org/apache/spark/rdd/RDDSuite.scala | 8 ++-- .../org/apache/spark/rdd/SortingSuite.scala | 1 - .../scheduler/ClusterSchedulerSuite.scala | 8 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 3 +- .../spark/scheduler/JobLoggerSuite.scala | 6 --- .../spark/scheduler/TaskContextSuite.scala | 7 +-- .../scheduler/TaskResultGetterSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 3 +- .../serializer/KryoSerializerSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 7 ++- .../spark/storage/DiskBlockManagerSuite.scala | 3 +- .../scala/org/apache/spark/ui/UISuite.scala | 6 ++- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- .../spark/util/ClosureCleanerSuite.scala | 4 +- .../apache/spark/util/NextIteratorSuite.scala | 6 ++- .../spark/util/SizeEstimatorSuite.scala | 3 +- .../org/apache/spark/util/UtilsSuite.scala | 10 +++-- .../util/collection/AppendOnlyMapSuite.scala | 3 +- .../spark/util/collection/BitSetSuite.scala | 1 - .../util/collection/OpenHashMapSuite.scala | 2 + .../util/collection/OpenHashSetSuite.scala | 1 - .../PrimitiveKeyOpenHashMapSuite.scala | 2 + .../util/random/RandomSamplerSuite.scala | 6 +-- .../util/random/XORShiftRandomSuite.scala | 1 + 246 files changed, 446 insertions(+), 552 deletions(-) diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index d2d778b7567bc..0d31894d6ec7a 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -17,6 +17,8 @@ package org.apache.spark.network.netty; +import java.util.concurrent.TimeUnit; + import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelOption; @@ -27,8 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.TimeUnit; - class FileClient { private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index 3ac045f9444f2..c0133e19c7f79 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -23,11 +23,11 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; import io.netty.channel.DefaultFileRegion; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.spark.storage.BlockId; import org.apache.spark.storage.FileSegment; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; class FileServerHandler extends SimpleChannelInboundHandler { diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index 32429f01acab8..1fca5729c6092 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -17,7 +17,8 @@ package org.apache.hadoop.mapreduce -import java.lang.{Integer => JInteger, Boolean => JBoolean} +import java.lang.{Boolean => JBoolean, Integer => JInteger} + import org.apache.hadoop.conf.Configuration private[apache] diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index df01b2e942180..73dd471ab1d52 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -19,8 +19,9 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} -import scala.collection.mutable.Map import scala.collection.generic.Growable +import scala.collection.mutable.Map + import org.apache.spark.serializer.JavaSerializer /** diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index d9ed572da6dba..754b46a4c7df2 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -20,12 +20,11 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator - private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[T]( diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index b38af2497d3d9..1daabecf23292 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -18,9 +18,9 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} -import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, RDDBlockId} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} /** Spark class responsible for passing RDDs split contents to the BlockManager and making sure a node doesn't load two copies of an RDD at once. diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index d7d10285dadcb..f2decd14ef6d9 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,10 +21,8 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try -import org.apache.spark.scheduler.{JobSucceeded, JobWaiter} -import org.apache.spark.scheduler.JobFailed import org.apache.spark.rdd.RDD - +import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** * A future for the result of an action to support cancellation. This is an extension of the diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index a885898ad48d4..d3264a4bb3c81 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -17,8 +17,10 @@ package org.apache.spark -import java.io.{File} +import java.io.File + import com.google.common.io.Files + import org.apache.spark.util.Utils private[spark] class HttpFileServer extends Logging { diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 69a738dc4446a..759e68ee0cc61 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -18,7 +18,6 @@ package org.apache.spark import java.io.File -import java.net.InetAddress import org.eclipse.jetty.server.Server import org.eclipse.jetty.server.bio.SocketConnector @@ -26,6 +25,7 @@ import org.eclipse.jetty.server.handler.DefaultHandler import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.server.handler.ResourceHandler import org.eclipse.jetty.util.thread.QueuedThreadPool + import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8d6db0fca23f7..5968973132942 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -22,7 +22,6 @@ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashSet import scala.concurrent.Await -import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index fdd4c24e2345f..dff665cae6cb6 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -19,9 +19,9 @@ package org.apache.spark import java.io._ +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable -import org.apache.hadoop.conf.Configuration class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala index a85aa50a9b94b..e8f756c408889 100644 --- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala @@ -17,10 +17,8 @@ package org.apache.spark -import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.Serializer - private[spark] abstract class ShuffleFetcher { /** diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 45d19bcbfa6f2..b947feb891ee6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -20,8 +20,6 @@ package org.apache.spark import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} - /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5a6d06b66e85b..a24f07e9a6e9a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,21 +19,18 @@ package org.apache.spark import java.io._ import java.net.URI -import java.util.{UUID, Properties} +import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger import scala.collection.{Map, Set} import scala.collection.generic.Growable - import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, - FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, - TextInputFormat} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, TextInputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary @@ -42,14 +39,12 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend, SimrSchedulerBackend} +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType, - ClosureCleaner} +import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6ae020f6a21b1..7ac65828f670f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -21,16 +21,15 @@ import scala.collection.mutable import scala.concurrent.Await import akka.actor._ +import com.google.common.collect.MapMaker +import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster} +import org.apache.spark.storage.{BlockManager, BlockManagerMaster, BlockManagerMasterActor} import org.apache.spark.network.ConnectionManager import org.apache.spark.serializer.{Serializer, SerializerManager} -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.api.python.PythonWorkerFactory - -import com.google.common.collect.MapMaker +import org.apache.spark.util.{AkkaUtils, Utils} /** * Holds all the runtime environment objects for a running Spark instance (either master or worker), diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 4e63117a51334..d404459a8eb7e 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -18,8 +18,8 @@ package org.apache.hadoop.mapred import java.io.IOException -import java.text.SimpleDateFormat import java.text.NumberFormat +import java.text.SimpleDateFormat import java.util.Date import org.apache.hadoop.fs.FileSystem diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 33737e1960ade..071044463d980 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -17,27 +17,25 @@ package org.apache.spark.api.java +import java.lang.{Double => JDouble} + import scala.reflect.ClassTag -import org.apache.spark.rdd.RDD +import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.util.StatCounter import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.StatCounter -import java.lang.Double -import org.apache.spark.Partitioner - -import scala.collection.JavaConverters._ - -class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { +class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { - override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]] + override val classTag: ClassTag[JDouble] = implicitly[ClassTag[JDouble]] - override val rdd: RDD[Double] = srdd.map(x => Double.valueOf(x)) + override val rdd: RDD[JDouble] = srdd.map(x => JDouble.valueOf(x)) - override def wrapRDD(rdd: RDD[Double]): JavaDoubleRDD = + override def wrapRDD(rdd: RDD[JDouble]): JavaDoubleRDD = new JavaDoubleRDD(rdd.map(_.doubleValue)) // Common RDD functions @@ -67,7 +65,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def unpersist(blocking: Boolean): JavaDoubleRDD = fromRDD(srdd.unpersist(blocking)) // first() has to be overriden here in order for its return type to be Double instead of Object. - override def first(): Double = srdd.first() + override def first(): JDouble = srdd.first() // Transformations (return a new RDD) @@ -84,7 +82,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: JFunction[Double, java.lang.Boolean]): JavaDoubleRDD = + def filter(f: JFunction[JDouble, java.lang.Boolean]): JavaDoubleRDD = fromRDD(srdd.filter(x => f(x).booleanValue())) /** @@ -133,7 +131,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaDoubleRDD = + def sample(withReplacement: Boolean, fraction: JDouble, seed: Int): JavaDoubleRDD = fromRDD(srdd.sample(withReplacement, fraction, seed)) /** @@ -145,7 +143,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav // Double RDD functions /** Add up the elements in this RDD. */ - def sum(): Double = srdd.sum() + def sum(): JDouble = srdd.sum() /** * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and @@ -154,35 +152,35 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def stats(): StatCounter = srdd.stats() /** Compute the mean of this RDD's elements. */ - def mean(): Double = srdd.mean() + def mean(): JDouble = srdd.mean() /** Compute the variance of this RDD's elements. */ - def variance(): Double = srdd.variance() + def variance(): JDouble = srdd.variance() /** Compute the standard deviation of this RDD's elements. */ - def stdev(): Double = srdd.stdev() + def stdev(): JDouble = srdd.stdev() /** * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev(): Double = srdd.sampleStdev() + def sampleStdev(): JDouble = srdd.sampleStdev() /** * Compute the sample variance of this RDD's elements (which corrects for bias in * estimating the standard variance by dividing by N-1 instead of N). */ - def sampleVariance(): Double = srdd.sampleVariance() + def sampleVariance(): JDouble = srdd.sampleVariance() /** Return the approximate mean of the elements in this RDD. */ - def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = + def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) /** (Experimental) Approximate operation to return the sum within a timeout. */ - def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = + def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) /** (Experimental) Approximate operation to return the sum within a timeout. */ @@ -222,7 +220,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav srdd.histogram(buckets, false) } - def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = { + def histogram(buckets: Array[JDouble], evenBuckets: Boolean): Array[Long] = { srdd.histogram(buckets.map(_.toDouble), evenBuckets) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 7b73057953552..0055c98844ded 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -20,8 +20,8 @@ package org.apache.spark.api.java import scala.reflect.ClassTag import org.apache.spark._ -import org.apache.spark.rdd.RDD import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java index 3e85052cd03e8..30e6a524742c4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java @@ -17,7 +17,6 @@ package org.apache.spark.api.java.function; - import java.io.Serializable; /** diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java index 5e9b8c48b89aa..490da255bc912 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java @@ -17,7 +17,6 @@ package org.apache.spark.api.java.function; - import java.io.Serializable; /** diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java index 537439ef53888..e0fcd460c8404 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -17,11 +17,10 @@ package org.apache.spark.api.java.function; -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; - import java.io.Serializable; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; /** * Base class for functions whose return types do not create special RDDs. PairFunction and diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java index a2d1214fb4692..16d7379462e15 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -17,11 +17,11 @@ package org.apache.spark.api.java.function; +import java.io.Serializable; + import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import java.io.Serializable; - /** * A two-argument function that takes arguments of type T1 and T2 and returns an R. */ diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java index fb1deceab5b56..096eb71f95cd0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java @@ -17,11 +17,10 @@ package org.apache.spark.api.java.function; +import java.io.Serializable; + import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction2; - -import java.io.Serializable; /** * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R. diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java index ca485b3cc2d93..c72b98c28a0cb 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -17,12 +17,12 @@ package org.apache.spark.api.java.function; +import java.io.Serializable; + import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import java.io.Serializable; - /** * A function that returns zero or more key-value pair records from each input record. The * key-value pairs are represented as scala.Tuple2 objects. diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java index cbe2306026a61..84b9136d98aa2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java @@ -17,12 +17,12 @@ package org.apache.spark.api.java.function; +import java.io.Serializable; + import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import java.io.Serializable; - /** * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. */ diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 35eca62ecd586..95bec5030bfdd 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -17,8 +17,6 @@ package org.apache.spark.api.python -import java.util.Arrays - import org.apache.spark.Partitioner import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 33667a998ed41..e4d0285710e84 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,9 +24,9 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast -import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index f291266fcf17c..a5f0f3d5e7eae 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,8 +17,8 @@ package org.apache.spark.api.python -import java.io.{OutputStreamWriter, File, DataInputStream, IOException} -import java.net.{ServerSocket, Socket, SocketException, InetAddress} +import java.io.{DataInputStream, File, IOException, OutputStreamWriter} +import java.net.{InetAddress, ServerSocket, Socket, SocketException} import scala.collection.JavaConversions._ diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 39ee0dbb92841..20207c261320b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv} +import org.apache.spark.{HttpServer, Logging, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index ec997255d59a1..22d783c8590c6 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -26,7 +26,6 @@ import org.apache.spark._ import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel} import org.apache.spark.util.Utils - private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 9987e2300ceb7..eb5676b51d836 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -23,13 +23,13 @@ import scala.concurrent._ import akka.actor._ import akka.pattern.ask +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.util.{AkkaUtils, Utils} -import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} /** * Proxy that relays messages to the driver. diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 7de7c4864ee54..190b331cfe7d8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -21,10 +21,10 @@ import java.io._ import java.net.URL import java.util.concurrent.TimeoutException +import scala.collection.mutable.ListBuffer import scala.concurrent.{Await, future, promise} -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global -import scala.collection.mutable.ListBuffer +import scala.concurrent.duration._ import scala.sys.process._ import net.liftweb.json.JsonParser diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 33e69371b8344..318beb5db5214 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -20,10 +20,9 @@ package org.apache.spark.deploy import net.liftweb.json.JsonDSL._ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} -import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo, DriverInfo} +import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner - private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 488843a32c167..a73b459c3cea1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -17,14 +17,14 @@ package org.apache.spark.deploy +import scala.collection.mutable.ArrayBuffer + import akka.actor.ActorSystem +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, Logging} - -import scala.collection.mutable.ArrayBuffer /** * Testing class that creates a Spark standalone process in-cluster (that is, running the diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 80179320325de..1550c3eb4286b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,9 +17,9 @@ package org.apache.spark.deploy.client -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{SparkConf, SparkContext, Logging} -import org.apache.spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.{ApplicationDescription, Command} +import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 3e26379166885..e8867bc1691d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -17,11 +17,14 @@ package org.apache.spark.deploy.master -import org.apache.spark.deploy.ApplicationDescription import java.util.Date -import akka.actor.ActorRef + import scala.collection.mutable +import akka.actor.ActorRef + +import org.apache.spark.deploy.ApplicationDescription + private[spark] class ApplicationInfo( val startTime: Long, val id: String, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 74bb9ebf1db4a..aa85aa060d9c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.io._ import akka.serialization.Serialization + import org.apache.spark.Logging /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index e44f90c1412e2..51794ce40cb45 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -30,15 +30,14 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension - -import org.apache.spark.{SparkConf, Logging, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.spark.deploy.master.DriverState.DriverState private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index e7f3224091ca8..a87781fb93850 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.master -import org.apache.spark.util.{Utils, IntParam} import org.apache.spark.SparkConf +import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 999090ad7464d..57758055b19c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.data.Stat -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf} /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 77c23fb9fbeea..47b8f67f8a45b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,7 +21,7 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.master.MasterMessages._ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 10816a1f43ff6..48b2fc06a9d70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,10 +17,10 @@ package org.apache.spark.deploy.master -import org.apache.spark.{SparkConf, Logging} +import akka.serialization.Serialization import org.apache.zookeeper._ -import akka.serialization.Serialization +import org.apache.spark.{Logging, SparkConf} class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index f29a6ad2e7b92..5cc4adbe448b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -17,11 +17,12 @@ package org.apache.spark.deploy.master.ui +import javax.servlet.http.HttpServletRequest + import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue import org.apache.spark.deploy.JsonProtocol diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 04f9a22a25a94..01c8f9065e50a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -17,12 +17,12 @@ package org.apache.spark.deploy.master.ui +import javax.servlet.http.HttpServletRequest + import scala.concurrent.Await -import scala.concurrent.duration._ import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 05c4df891ed75..5ab13e7aa6b1f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest + import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 2ceccc703d291..0c761dfc93a1f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.worker -import java.io.{File, FileOutputStream, IOException, InputStream} +import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 18885d7ca6daa..2edd921066876 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -20,12 +20,11 @@ package org.apache.spark.deploy.worker import java.io._ import akka.actor.ActorRef - import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.Logging -import org.apache.spark.deploy.{ExecutorState, ApplicationDescription, Command} +import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f4ee0e2343849..7b0b7861b76e1 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -31,7 +31,6 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 3ed528e6b3773..d35d5be73ff97 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -17,9 +17,10 @@ package org.apache.spark.deploy.worker -import org.apache.spark.util.{Utils, IntParam, MemoryParam} import java.lang.management.ManagementFactory +import org.apache.spark.util.{IntParam, MemoryParam, Utils} + /** * Command-line parser for the master. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 86688e44242a9..bdf126f93abc8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -18,11 +18,11 @@ package org.apache.spark.deploy.worker.ui import java.io.File - import javax.servlet.http.HttpServletRequest + import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 45b43b403dd8c..0aae569b17272 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index ad7dd34c76940..3d34960653f5d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer + import org.apache.spark.TaskState.TaskState /** diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index c2e973e1738d4..127f5e90f3e1a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -17,12 +17,11 @@ package org.apache.spark.executor -import com.codahale.metrics.{Gauge, MetricRegistry} +import scala.collection.JavaConversions._ +import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.hadoop.fs.FileSystem -import scala.collection.JavaConversions._ - import org.apache.spark.metrics.source.Source class ExecutorSource(val executor: Executor, executorId: String) extends Source { diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index b56d8c99124df..6fc702fdb1512 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -20,8 +20,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer import com.google.protobuf.ByteString - -import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver} +import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.Logging @@ -29,7 +28,6 @@ import org.apache.spark.TaskState import org.apache.spark.TaskState.TaskState import org.apache.spark.util.Utils - private[spark] class MesosExecutorBackend extends MesosExecutor with ExecutorBackend diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 59801773205bd..848b5c439bb5b 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -20,10 +20,9 @@ package org.apache.spark.io import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} -import org.apache.spark.{SparkEnv, SparkConf} +import org.apache.spark.SparkConf /** * CompressionCodec allows the customization of choosing different compression implementations diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index e54ac0b332093..6883a54494598 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -17,8 +17,8 @@ package org.apache.spark.metrics +import java.io.{FileInputStream, InputStream} import java.util.Properties -import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index de233e416a9dc..966c092124266 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -17,14 +17,14 @@ package org.apache.spark.metrics -import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} - import java.util.Properties import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.apache.spark.{SparkConf, Logging} +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} + +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index bce257d6e6f47..98fa1dbd7c6ab 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -17,11 +17,11 @@ package org.apache.spark.metrics.sink -import com.codahale.metrics.{ConsoleReporter, MetricRegistry} - import java.util.Properties import java.util.concurrent.TimeUnit +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + import org.apache.spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 3d1a06a395a72..40f64768e6885 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -17,12 +17,12 @@ package org.apache.spark.metrics.sink -import com.codahale.metrics.{CsvReporter, MetricRegistry} - import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit +import com.codahale.metrics.{CsvReporter, MetricRegistry} + import org.apache.spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index b924907070eb9..410ca0704b5c4 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -20,8 +20,8 @@ package org.apache.spark.metrics.sink import java.util.Properties import java.util.concurrent.TimeUnit -import com.codahale.metrics.ganglia.GangliaReporter import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.ganglia.GangliaReporter import info.ganglia.gmetric4j.gmetric.GMetric import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index cdcfec8ca785b..e09be001421fc 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -17,12 +17,12 @@ package org.apache.spark.metrics.sink +import java.net.InetSocketAddress import java.util.Properties import java.util.concurrent.TimeUnit -import java.net.InetSocketAddress import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.graphite.{GraphiteReporter, Graphite} +import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 621d086d415cc..b5cf210af2119 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -17,10 +17,10 @@ package org.apache.spark.metrics.sink -import com.codahale.metrics.{JmxReporter, MetricRegistry} - import java.util.Properties +import com.codahale.metrics.{JmxReporter, MetricRegistry} + class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 99357fede6d06..3cdfe26d40f66 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -17,15 +17,13 @@ package org.apache.spark.metrics.sink -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.json.MetricsModule - -import com.fasterxml.jackson.databind.ObjectMapper - import java.util.Properties import java.util.concurrent.TimeUnit import javax.servlet.http.HttpServletRequest +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.json.MetricsModule +import com.fasterxml.jackson.databind.ObjectMapper import org.eclipse.jetty.server.Handler import org.apache.spark.ui.JettyUtils diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala index fb4c65909a9e2..d3c09b16063d6 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala @@ -23,7 +23,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.storage.BlockManager - private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) extends Message(Message.BUFFER_MESSAGE, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index ae2007e41b77f..f2e3c1a14ecc6 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -17,16 +17,13 @@ package org.apache.spark.network -import org.apache.spark._ - -import scala.collection.mutable.{HashMap, Queue, ArrayBuffer} - -import java.io._ +import java.net._ import java.nio._ import java.nio.channels._ -import java.nio.channels.spi._ -import java.net._ +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} + +import org.apache.spark._ private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector, diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index a78d6ac70f8ca..3dd82bee0b5fd 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,24 +17,21 @@ package org.apache.spark.network -import org.apache.spark._ - +import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ -import java.net._ import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} -import scala.collection.mutable.HashSet +import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import scala.collection.mutable.SynchronizedMap import scala.collection.mutable.SynchronizedQueue -import scala.collection.mutable.ArrayBuffer - -import scala.concurrent.{Await, Promise, ExecutionContext, Future} -import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ +import org.apache.spark._ import org.apache.spark.util.Utils private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala index 50dd9bc2d101f..b82edb6850d23 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala @@ -21,7 +21,6 @@ import java.net.InetSocketAddress import org.apache.spark.util.Utils - private[spark] case class ConnectionManagerId(host: String, port: Int) { // DEBUG code Utils.checkHost(host) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 8e5c5296cb8d1..35f64134b073a 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -17,16 +17,13 @@ package org.apache.spark.network -import org.apache.spark._ -import org.apache.spark.SparkContext._ - -import scala.io.Source - import java.nio.ByteBuffer -import java.net.InetAddress import scala.concurrent.Await import scala.concurrent.duration._ +import scala.io.Source + +import org.apache.spark._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala index 2612884bdbe15..20fe67661844f 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/Message.scala @@ -17,12 +17,11 @@ package org.apache.spark.network -import java.nio.ByteBuffer import java.net.InetSocketAddress +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer - private[spark] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null var started = false diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala index e0fe57b80d5cd..d0f986a12bfe0 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala @@ -21,7 +21,6 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer - private[network] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala index 235fbc39b3bd2..9bcbc6141a502 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala @@ -21,7 +21,6 @@ import java.net.InetAddress import java.net.InetSocketAddress import java.nio.ByteBuffer - private[spark] class MessageChunkHeader( val typ: Long, val id: Int, diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 1c9d6030d68d7..9976255c7e251 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import java.net.InetAddress + import org.apache.spark.SparkConf private[spark] object ReceiverTest { diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 162d49bf61617..646f8425d9551 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import java.net.InetAddress + import org.apache.spark.SparkConf private[spark] object SenderTest { diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 1b9fa1e53a9f7..f9082ffb9141a 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -20,7 +20,7 @@ package org.apache.spark.network.netty import io.netty.buffer._ import org.apache.spark.Logging -import org.apache.spark.storage.{TestBlockId, BlockId} +import org.apache.spark.storage.{BlockId, TestBlockId} private[spark] class FileHeader ( val fileLen: Int, diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index d87157e12c487..e7b2855e1ec91 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -19,17 +19,16 @@ package org.apache.spark.network.netty import java.util.concurrent.Executors +import scala.collection.JavaConverters._ + import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.util.CharsetUtil -import org.apache.spark.{SparkContext, SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.network.ConnectionManagerId - -import scala.collection.JavaConverters._ import org.apache.spark.storage.BlockId - private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { def getBlock(host: String, port: Int, blockId: BlockId, diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala index 44204a8c46572..7ef7aecc6a9fb 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala @@ -23,7 +23,6 @@ import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.storage.{BlockId, FileSegment} - private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { val server = new FileServer(pResolver, portIn) diff --git a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala index 423ff67a5fd43..d25452daf7606 100644 --- a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala @@ -18,8 +18,8 @@ package org.apache.spark.partial import org.apache.spark._ -import org.apache.spark.scheduler.JobListener import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.JobListener /** * A JobListener for an approximate single-result action, such as count() or non-parallel reduce(). diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index e519e3a54846e..40b70baabcad9 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -18,14 +18,12 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import java.util.{Map => JMap} +import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.Map import scala.collection.mutable.HashMap -import scala.collection.JavaConversions.mapAsScalaMap import cern.jet.stat.Probability - import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} /** diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index cf8a5680b663d..b5111891ed35b 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -18,11 +18,10 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import java.util.{Map => JMap} -import scala.collection.mutable.HashMap -import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.Map +import scala.collection.mutable.HashMap import org.apache.spark.util.StatCounter diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala index 8225a5d933ce5..442fb86227d86 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala @@ -18,11 +18,10 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import java.util.{Map => JMap} -import scala.collection.mutable.HashMap -import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.Map +import scala.collection.mutable.HashMap import org.apache.spark.util.StatCounter diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 424354ae165a0..e6c4a6d3794a0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} import org.apache.spark.storage.{BlockId, BlockManager} private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 87b950ba431f9..4908711d17db7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -17,10 +17,11 @@ package org.apache.spark.rdd -import java.io.{ObjectOutputStream, IOException} +import java.io.{IOException, ObjectOutputStream} + import scala.reflect.ClassTag -import org.apache.spark._ +import org.apache.spark._ private[spark] class CartesianPartition( diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 8f9d1d5a84c36..888af541cf970 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -18,12 +18,15 @@ package org.apache.spark.rdd import java.io.IOException + import scala.reflect.ClassTag + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 0e47f2e022610..699a10c96c227 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.io.{ObjectOutputStream, IOException} +import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index dc345b2df079b..4e82b51313bf0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -17,13 +17,14 @@ package org.apache.spark.rdd -import org.apache.spark._ -import java.io.{ObjectOutputStream, IOException} +import java.io.{IOException, ObjectOutputStream} + import scala.collection.mutable -import scala.Some import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag +import org.apache.spark._ + /** * Class that captures a coalesced RDD by essentially keeping track of parent partitions * @param index of this coalesced partition diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 20713b4249b5e..a7b6b3b5146ce 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,14 +17,12 @@ package org.apache.spark.rdd +import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.partial.SumEvaluator import org.apache.spark.util.StatCounter -import org.apache.spark.{TaskContext, Logging} - -import scala.collection.immutable.NumericRange /** * Extra functions available on RDDs of Doubles through an implicit conversion. diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index e74c83b90baa3..9e41b3d1e2d4f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{OneToOneDependency, Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class FilteredRDD[T: ClassTag]( prev: RDD[T], f: T => Boolean) diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala index 4d1878fc142ac..d8f87d4e3690e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -17,9 +17,9 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} private[spark] class FlatMappedRDD[U: ClassTag, T: ClassTag]( diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala index 82000bac092da..7c9023f62d3b6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala @@ -17,8 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{TaskContext, Partition} - +import org.apache.spark.{Partition, TaskContext} private[spark] class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U]) diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala index 1a694475f6975..f6463fa715a71 100644 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) extends RDD[Array[T]](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index ad74d4636fb1b..a374fc4a871b0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.io.EOFException -import org.apache.hadoop.conf.{Configuration, Configurable} +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf @@ -32,7 +32,6 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator - /** * A Spark split class that wraps around a Hadoop InputSplit. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index db15baf503ad6..4883fb828814c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index d33c1af58154c..2bc47eb9fcd74 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -17,8 +17,7 @@ package org.apache.spark.rdd - -import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.{Partition, TaskContext} private[spark] class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 10d519e6971f1..15bec39659e17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -22,12 +22,13 @@ import java.text.SimpleDateFormat import java.util.Date import java.util.{HashMap => JHashMap} +import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ -import scala.reflect.{ClassTag, classTag} +import scala.reflect.ClassTag +import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.SequenceFile.CompressionType @@ -38,15 +39,14 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import com.clearspring.analytics.stream.cardinality.HyperLogLog - // SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark. import org.apache.hadoop.mapred.SparkHadoopWriter import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil + import org.apache.spark._ +import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} -import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.util.SerializableHyperLogLog /** diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index f270c1ac21757..5f03d7d650a30 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -17,14 +17,15 @@ package org.apache.spark.rdd +import java.io._ + +import scala.Serializable +import scala.collection.Map import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer -import scala.collection.Map import scala.reflect.ClassTag import org.apache.spark._ -import java.io._ -import scala.Serializable import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index ea8885b36e3ae..b0440ca7f32cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -19,8 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{NarrowDependency, SparkEnv, Partition, TaskContext} - +import org.apache.spark.{NarrowDependency, Partition, TaskContext} class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { override val index = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index f4364329a3a71..a84357b38414e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -17,10 +17,11 @@ package org.apache.spark.rdd +import java.io.{IOException, ObjectOutputStream} + import scala.reflect.ClassTag -import java.io.{ObjectOutputStream, IOException} -import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition} +import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} /** * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index a74309d861318..ce4c0d382baab 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -21,7 +21,7 @@ import java.util.Random import scala.reflect.ClassTag -import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.random.RandomSampler private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 8ef919c4b58cb..abd4414e81f5c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -20,14 +20,13 @@ package org.apache.spark.rdd import java.io.PrintWriter import java.util.StringTokenizer -import scala.collection.Map import scala.collection.JavaConversions._ +import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import org.apache.spark.{SparkEnv, Partition, TaskContext} - +import org.apache.spark.{Partition, SparkEnv, TaskContext} /** * An RDD that pipes the contents of each parent partition through an external command diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index d4fc28f55137b..50320f40350cd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -22,30 +22,27 @@ import java.util.Random import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer - import scala.reflect.{classTag, ClassTag} +import com.clearspring.analytics.stream.cardinality.HyperLogLog +import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.TextOutputFormat -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} -import com.clearspring.analytics.stream.cardinality.HyperLogLog - +import org.apache.spark._ import org.apache.spark.Partitioner._ +import org.apache.spark.SparkContext._ import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableHyperLogLog} - -import org.apache.spark.SparkContext._ -import org.apache.spark._ -import org.apache.spark.util.random.{PoissonSampler, BernoulliSampler} +import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 73e8769c0981d..953f0555e57c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -20,9 +20,8 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.hadoop.fs.Path -import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SerializableWritable, Partition, SparkException, Logging} +import org.apache.spark.{Logging, Partition, SerializableWritable, SparkException} import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 08534b6f1db3e..b50307cfa49b7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import scala.reflect.ClassTag import java.util.Random +import scala.reflect.ClassTag + import cern.jet.random.Poisson import cern.jet.random.engine.DRand diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index c9b4c768a98b4..7df9a2960d8a5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -16,15 +16,15 @@ */ package org.apache.spark.rdd -import scala.reflect.{ ClassTag, classTag} +import scala.reflect.{ClassTag, classTag} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.io.Writable -import org.apache.spark.SparkContext._ import org.apache.spark.Logging +import org.apache.spark.SparkContext._ /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 0ccb309d0d9d5..0bbda25a905cd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -19,8 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, - SparkEnv, TaskContext} +import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 4f90c7d3d68ab..5fe9f363db453 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -23,14 +23,13 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import org.apache.spark.Partitioner import org.apache.spark.Dependency -import org.apache.spark.TaskContext +import org.apache.spark.OneToOneDependency import org.apache.spark.Partition -import org.apache.spark.SparkEnv +import org.apache.spark.Partitioner import org.apache.spark.ShuffleDependency -import org.apache.spark.OneToOneDependency - +import org.apache.spark.SparkEnv +import org.apache.spark.TaskContext /** * An optimized version of cogroup for set difference/subtraction. diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 08a41ac5583d5..a447030752096 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -17,12 +17,12 @@ package org.apache.spark.rdd +import java.io.{IOException, ObjectOutputStream} + import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import org.apache.spark.{Dependency, RangeDependency, SparkContext, Partition, TaskContext} - -import java.io.{ObjectOutputStream, IOException} +import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 83be3c6eb40c4..b56643444aa40 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -17,10 +17,12 @@ package org.apache.spark.rdd -import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} -import java.io.{ObjectOutputStream, IOException} +import java.io.{IOException, ObjectOutputStream} + import scala.reflect.ClassTag +import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} + private[spark] class ZippedPartitionsPartition( idx: Int, @transient rdds: Seq[RDD[_]], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index fb5b070c18d43..2119e76f0e032 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -17,12 +17,12 @@ package org.apache.spark.rdd -import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} - -import java.io.{ObjectOutputStream, IOException} +import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag +import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} + private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( idx: Int, @transient rdd1: RDD[T], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index 38dc114d80812..e2c301603b4a5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.Utils private[spark] diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index 0b04607d019a8..9257f48559c9e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -17,10 +17,10 @@ package org.apache.spark.scheduler -import org.apache.spark.TaskContext - import java.util.Properties +import org.apache.spark.TaskContext + /** * Tracks information about an active job in the DAGScheduler. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 80211541a6a63..729f518b89c06 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -28,9 +28,9 @@ import scala.reflect.ClassTag import akka.actor._ import org.apache.spark._ -import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index add11876130b1..39cd98e2d74e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -22,8 +22,8 @@ import java.util.Properties import scala.collection.mutable.Map import org.apache.spark._ -import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics +import org.apache.spark.rdd.RDD /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 7b5c0e29ad840..b52fe2410abde 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import com.codahale.metrics.{Gauge,MetricRegistry} -import org.apache.spark.metrics.source.Source import org.apache.spark.SparkContext +import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 23447f1bbf852..5555585c8b4cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -17,17 +17,17 @@ package org.apache.spark.scheduler -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.deploy.SparkHadoopUtil +import scala.collection.JavaConversions._ import scala.collection.immutable.Set +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.conf.Configuration -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.collection.JavaConversions._ +import org.apache.hadoop.util.ReflectionUtils +import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil /** * Parses and holds information about inputFormat (and files) specified as a parameter. diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index b909b66a5de76..9d75d7c4ad69a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{IOException, File, FileNotFoundException, PrintWriter} +import java.io.{File, FileNotFoundException, IOException, PrintWriter} import java.text.SimpleDateFormat import java.util.{Date, Properties} import java.util.concurrent.LinkedBlockingQueue @@ -25,8 +25,8 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import org.apache.spark._ -import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 1c61687f28006..d3f63ff92ac6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -17,8 +17,9 @@ package org.apache.spark.scheduler +import java.io.{Externalizable, ObjectInput, ObjectOutput} + import org.apache.spark.storage.BlockManagerId -import java.io.{ObjectOutput, ObjectInput, Externalizable} /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 77b1682b3e47c..3fc6cc9850feb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -23,7 +23,7 @@ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDDCheckpointData -import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] object ResultTask { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index d573e125a33d1..ed24eb6a549dd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import scala.collection.mutable.ArrayBuffer /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index a546193d5b49a..e4eced383c3a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -20,10 +20,10 @@ package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} import java.util.{NoSuchElementException, Properties} -import org.apache.spark.{SparkConf, Logging} - import scala.xml.XML +import org.apache.spark.{Logging, SparkConf} + /** * An interface to build Schedulable tree * buildPools: build the tree nodes(pools) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 02bdbba825781..eefc8c232b564 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.SparkContext - /** * A backend interface for scheduling systems that allows plugging in different ones under * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index a37ead563271a..77789031f464a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -24,11 +24,10 @@ import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.storage._ -import org.apache.spark.util.{MetadataCleanerType, TimeStampedHashMap, MetadataCleaner} import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDDCheckpointData - +import org.apache.spark.storage._ +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] object ShuffleMapTask { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 129153c732d9a..9590c03f10632 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -18,9 +18,10 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.util.{Utils, Distribution} + import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.{Distribution, Utils} sealed trait SparkListenerEvents diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 69b42e86eae3e..b85b4a50cd93a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -29,7 +29,6 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream - /** * A unit of execution. We have two kinds of Task's in Spark: * - [[org.apache.spark.scheduler.ShuffleMapTask]] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 5190d234d4ee5..1481d70db42e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer + import org.apache.spark.util.SerializableBuffer private[spark] class TaskDescription( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 91c27d7b8e9d7..6183b125def99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.util.Utils - /** * Information about a running task attempt inside a TaskSet. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 35de13c385187..ea3229b75be36 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler - private[spark] object TaskLocality extends Enumeration { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 5724ec9d1b4d7..d49d8fb887007 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -18,13 +18,14 @@ package org.apache.spark.scheduler import java.io._ +import java.nio.ByteBuffer import scala.collection.mutable.Map + +import org.apache.spark.SparkEnv import org.apache.spark.executor.TaskMetrics -import org.apache.spark.{SparkEnv} -import java.nio.ByteBuffer -import org.apache.spark.util.Utils import org.apache.spark.storage.BlockId +import org.apache.spark.util.Utils // Task result. Also contains updates to accumulator variables. private[spark] sealed trait TaskResult[T] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index bdec08e968a45..cb4ad4ae9350c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit} import org.apache.spark._ import org.apache.spark.TaskState.TaskState diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5b525155e9f62..8df37c247d0d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -18,13 +18,13 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} +import java.util.concurrent.atomic.AtomicLong +import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 21b2ff1682b78..1a4b7e599c01e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -26,13 +26,11 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import org.apache.spark.{ExceptionFailure, ExecutorLostFailure, FetchFailed, Logging, Resubmitted, - SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} +import org.apache.spark.{ExceptionFailure, ExecutorLostFailure, FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.{Clock, SystemClock} - /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of * each task, retries tasks if they fail (up to a limited number of times), and diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 53316dae2a6c8..4a9a1659d8254 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -21,8 +21,7 @@ import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.TaskDescription -import org.apache.spark.util.{Utils, SerializableBuffer} - +import org.apache.spark.util.{SerializableBuffer, Utils} private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 78204103a9bbd..379e02eb9a437 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,10 +27,8 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.{Logging, SparkException, TaskState} -import org.apache.spark.scheduler.{TaskSchedulerImpl, SchedulerBackend, SlaveLost, TaskDescription, - WorkerOffer} +import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 04f35cca08262..ee4b65e312abc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,11 +17,9 @@ package org.apache.spark.scheduler.cluster -import scala.collection.mutable.HashMap - import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 4401f6df47421..28b019d9fd495 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -18,18 +18,17 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{ArrayList => JArrayList, List => JList} +import java.util.{List => JList} import java.util.Collections -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, HashSet} -import com.google.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Logging, SparkContext, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index fef291eea0257..c576beb0c0d38 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -21,17 +21,16 @@ import java.io.File import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import com.google.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkException, SparkContext, TaskState} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, - TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} +import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 897d47a9ad981..50f7e79e97dd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 5d3d43623d9d2..33c1705ad7c58 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -20,8 +20,8 @@ package org.apache.spark.serializer import java.io._ import java.nio.ByteBuffer -import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.SparkConf +import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { val objOut = new ObjectOutputStream(out) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 2d0b25538505c..920490f9d0d61 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -17,13 +17,13 @@ package org.apache.spark.serializer -import java.nio.ByteBuffer import java.io.{EOFException, InputStream, OutputStream} +import java.nio.ByteBuffer -import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import com.esotericsoftware.kryo.{KryoException, Kryo} +import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} -import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} +import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} +import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index a38a2b59dbc23..16677ab54be04 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,8 +22,7 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{NextIterator, ByteBufferInputStream} - +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** * A serializer. Because some serialization libraries are not thread safe, this class is used to diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 36a37af4f821d..65ac0155f45e7 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -18,8 +18,8 @@ package org.apache.spark.serializer import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.SparkConf +import org.apache.spark.SparkConf /** * A service that returns a serializer object given the serializer's class name. If a previous diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index aa62ab5aba1c2..925022e7fe6fb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -17,7 +17,6 @@ package org.apache.spark.storage -import java.nio.ByteBuffer import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.ArrayBuffer @@ -26,15 +25,13 @@ import scala.collection.mutable.Queue import io.netty.buffer.ByteBuf -import org.apache.spark.Logging -import org.apache.spark.SparkException +import org.apache.spark.{Logging, SparkException} import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils - /** * A block fetcher iterator interface. There are two implementations: * diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 780a3a15dd15b..a734ddc1ef702 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -20,24 +20,21 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.collection.mutable.{HashMap, ArrayBuffer} -import scala.util.Random - -import akka.actor.{ActorSystem, Cancellable, Props} +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} -import scala.concurrent.duration.Duration import scala.concurrent.duration._ +import scala.util.Random +import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} +import sun.nio.ch.DirectBuffer -import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -import sun.nio.ch.DirectBuffer - private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 74207f59af170..98cd6e68fa724 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap + import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index c54e4f2664753..e531467cccb40 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,7 +23,7 @@ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask -import org.apache.spark.{SparkConf, Logging, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 893418fb8cad9..a999d76a326a6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import org.apache.spark.{SparkConf, Logging, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 45f51da288548..bbb9529b5a0ca 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -21,7 +21,6 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef - private[storage] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// // Messages from the master to slaves. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 3a65e55733834..bcfb82d3c7336 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -21,7 +21,6 @@ import akka.actor.Actor import org.apache.spark.storage.BlockManagerMessages._ - /** * An actor to take commands from the master to execute options. For example, * this is used to remove blocks from the slave's BlockManager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 7cf754fb204c2..687586490abfe 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -19,9 +19,8 @@ package org.apache.spark.storage import com.codahale.metrics.{Gauge,MetricRegistry} -import org.apache.spark.metrics.source.Source import org.apache.spark.SparkContext - +import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index 3efe738a08f66..c7766a3a65671 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.nio.ByteBuffer -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.network._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index fbafcf79d2833..7168ae18c2615 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -19,8 +19,8 @@ package org.apache.spark.storage import java.nio.ByteBuffer -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.StringBuilder import org.apache.spark.network._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index ea426562402ae..b047644b88f48 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.nio.ByteBuffer + import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 5a1e7b44440fd..d1f07ddb24bb2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -27,7 +27,6 @@ import org.apache.spark.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils - /** * Stores BlockManager blocks on disk. */ diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index eb5a18521683e..18141756518c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -17,10 +17,11 @@ package org.apache.spark.storage -import java.util.LinkedHashMap -import java.util.concurrent.ArrayBlockingQueue import java.nio.ByteBuffer -import collection.mutable.ArrayBuffer +import java.util.LinkedHashMap + +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.{SizeEstimator, Utils} /** diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index 40734aab49f93..8cea302eb14c3 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -17,11 +17,11 @@ package org.apache.spark.storage -import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{CountDownLatch, Executors} +import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.serializer.KryoSerializer import org.apache.spark.SparkContext +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 50a0cdb3095cd..2d88a40fbb3f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,8 +17,8 @@ package org.apache.spark.storage -import org.apache.spark.{SparkContext} -import BlockManagerMasterActor.BlockStatus +import org.apache.spark.SparkContext +import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.util.Utils private[spark] diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 729ba2c550a20..1d81d006c0b29 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -17,12 +17,13 @@ package org.apache.spark.storage -import akka.actor._ - import java.util.concurrent.ArrayBlockingQueue + +import akka.actor._ import util.Random + +import org.apache.spark.SparkConf import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.{SparkConf, SparkContext} /** * This class tests the BlockManager and MemoryStore for thread safety and diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index ade8ba1323b80..1f048a84cdfb6 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,21 +17,19 @@ package org.apache.spark.ui +import java.net.InetSocketAddress import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec -import scala.util.{Try, Success, Failure} +import scala.util.{Failure, Success, Try} import scala.xml.Node import net.liftweb.json.{JValue, pretty, render} - -import org.eclipse.jetty.server.{Server, Request, Handler} -import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import org.eclipse.jetty.server.{Handler, Request, Server} +import org.eclipse.jetty.server.handler.{AbstractHandler, ContextHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool import org.apache.spark.Logging -import java.net.InetSocketAddress - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 0196f43d7431b..af6b65860e006 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,16 +17,14 @@ package org.apache.spark.ui -import javax.servlet.http.HttpServletRequest - import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.{Logging, SparkContext, SparkEnv} +import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI -import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI -import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark */ diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index f913ee461b219..18d2b5075aa08 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -23,7 +23,6 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.SchedulingMode - /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 88f41be8d3dd2..9e7cdc88162e8 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -25,11 +25,10 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler +import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.UIUtils import org.apache.spark.ui.Page.Environment -import org.apache.spark.SparkContext - +import org.apache.spark.ui.UIUtils private[spark] class EnvironmentUI(sc: SparkContext) { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 4e41acf0230f9..1f3b7a4c231b6 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -26,14 +26,13 @@ import org.eclipse.jetty.server.Handler import org.apache.spark.{ExceptionFailure, Logging, SparkContext} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.scheduler.TaskInfo import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils - private[spark] class ExecutorsUI(val sc: SparkContext) { private var _listener: Option[ExecutorsListener] = None diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index ab03eb5ce1ab4..d012ba4dbb3db 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -17,11 +17,11 @@ package org.apache.spark.ui.jobs +import scala.collection.mutable import scala.xml.Node import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.util.Utils -import scala.collection.mutable /** Page showing executor summary */ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 6289f8744f240..81713edcf5db2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -25,7 +25,6 @@ import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils._ - /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 858a10ce750ff..07a08f5277d19 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} import org.apache.spark.{ExceptionFailure, SparkContext, Success} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index c1ee2f3d00d66..557bce6b66353 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -17,23 +17,15 @@ package org.apache.spark.ui.jobs -import scala.concurrent.duration._ - import java.text.SimpleDateFormat - import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler - import scala.Seq -import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} +import org.eclipse.jetty.server.Handler + +import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.{ExceptionFailure, SparkContext, Success} -import org.apache.spark.scheduler._ -import collection.mutable -import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 89fffcb80d0d7..eb7518a020840 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -19,12 +19,10 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} -import scala.collection.mutable.HashSet +import scala.xml.Node -import org.apache.spark.scheduler.Stage -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils._ /** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b6e98942ab811..ddc687a45a095 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -18,17 +18,16 @@ package org.apache.spark.ui.jobs import java.util.Date - import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.{ExceptionFailure} +import org.apache.spark.ExceptionFailure import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.TaskInfo import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.TaskInfo /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 999a94fc2d008..c5fd3ae16dc9f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -19,14 +19,13 @@ package org.apache.spark.ui.jobs import java.util.Date -import scala.xml.Node import scala.collection.mutable.HashSet +import scala.xml.Node import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils - /** Page showing list of all ongoing and recently finished stages */ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 39f422dd6b90f..dc18eab74e0da 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -17,8 +17,6 @@ package org.apache.spark.ui.storage -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 109a7d4094c0a..6a3c41fb1155d 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{RDDInfo, StorageUtils} -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils._ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index b83cd54f3c39a..78b149b14b1d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -23,11 +23,10 @@ import scala.xml.Node import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils} import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils._ import org.apache.spark.util.Utils - /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { val sc = parent.sc diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 761d378c7fd8b..f26ed47e58046 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -22,8 +22,8 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory - import org.apache.log4j.{Level, Logger} + import org.apache.spark.SparkConf /** diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index a38329df037a1..c3692f2fd929b 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -19,8 +19,9 @@ package org.apache.spark.util import java.io.Serializable import java.util.{PriorityQueue => JPriorityQueue} -import scala.collection.generic.Growable + import scala.collection.JavaConverters._ +import scala.collection.generic.Growable /** * Bounded priority queue. This class wraps the original PriorityQueue diff --git a/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala index e214d2a519a20..54de4d4ee8ca7 100644 --- a/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.io.InputStream import java.nio.ByteBuffer + import org.apache.spark.storage.BlockManager /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index c0c057be8defc..681d0a30cb3f8 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -17,14 +17,14 @@ package org.apache.spark.util -import java.lang.reflect.Field +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.objectweb.asm.Opcodes._ -import java.io.{ByteArrayOutputStream, ByteArrayInputStream} + import org.apache.spark.Logging private[spark] object ClosureCleaner extends Logging { diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 3868ab36312a7..0448919e09161 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -17,9 +17,9 @@ package org.apache.spark.util -import java.util.{TimerTask, Timer} -import org.apache.spark.{SparkConf, Logging} +import java.util.{Timer, TimerTask} +import org.apache.spark.{Logging, SparkConf} /** * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 34f1f6606fc3f..b053266f12748 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,7 +17,6 @@ package org.apache.spark.util - /** * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index f2b1ad7d0e91d..2b452ad33b021 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -17,8 +17,8 @@ package org.apache.spark.util +import java.io.{EOFException, IOException, ObjectInputStream, ObjectOutputStream} import java.nio.ByteBuffer -import java.io.{IOException, ObjectOutputStream, EOFException, ObjectInputStream} import java.nio.channels.Channels /** diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala index 2110b3596ee2a..21a88eea3bbc2 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala @@ -17,8 +17,9 @@ package org.apache.spark.util -import java.io.{Externalizable, ObjectOutput, ObjectInput} -import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog} +import java.io.{Externalizable, ObjectInput, ObjectOutput} + +import com.clearspring.analytics.stream.cardinality.{HyperLogLog, ICardinality} /** * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 17c6481c18463..b955612ca7749 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -17,20 +17,19 @@ package org.apache.spark.util +import java.lang.management.ManagementFactory +import java.lang.reflect.{Array => JArray} import java.lang.reflect.Field import java.lang.reflect.Modifier -import java.lang.reflect.{Array => JArray} import java.util.IdentityHashMap -import java.util.concurrent.ConcurrentHashMap import java.util.Random - -import javax.management.MBeanServer -import java.lang.management.ManagementFactory +import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging} + +import org.apache.spark.Logging /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 8e07a0f29addf..ddbd084ed7f01 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -18,10 +18,11 @@ package org.apache.spark.util import java.util.concurrent.ConcurrentHashMap + import scala.collection.JavaConversions -import scala.collection.mutable.Map import scala.collection.immutable -import org.apache.spark.scheduler.MapStatus +import scala.collection.mutable.Map + import org.apache.spark.Logging /** diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 26983138ff0da..19bece86b36b4 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -17,10 +17,10 @@ package org.apache.spark.util -import scala.collection.mutable.Set -import scala.collection.JavaConversions import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConversions +import scala.collection.mutable.Set class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8749ab7875bb7..8e69f1d3351b5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,7 +18,8 @@ package org.apache.spark.util import java.io._ -import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} +import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL} +import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} @@ -30,16 +31,11 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} -import org.apache.hadoop.io._ - +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil -import java.nio.ByteBuffer -import org.apache.spark.{SparkConf, SparkException, Logging} - /** * Various utility methods used by Spark. diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 96da93d854577..d437c055f33d4 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import scala.util.Random + import org.apache.spark.util.random.XORShiftRandom class Vector(val elements: Array[Double]) extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index c9cf512843875..d3153d2cac4a5 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -17,7 +17,6 @@ package org.apache.spark.util.collection - /** * A simple, fixed-size bit set implementation. This implementation is fast because it avoids * safety/bound checking. diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 59ba1e457ce11..856d092ab3463 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -23,8 +23,8 @@ import java.util.Comparator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import com.google.common.io.ByteStreams +import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 6b66d54751987..0f1fca4813ba9 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.random import java.util.Random + import cern.jet.random.Poisson import cern.jet.random.engine.DRand diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 20d32d01b5e19..ca611b67ed91d 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.random import java.util.{Random => JavaRandom} + import org.apache.spark.util.Utils.timeIt /** diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index c443c5266e42b..6c73ea6949dd2 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark +import scala.collection.mutable + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import collection.mutable -import java.util.Random -import scala.math.exp -import scala.math.signum + import org.apache.spark.SparkContext._ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index ec13b329b25a8..d2e29f20f0b08 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark +import java.io.File + import scala.reflect.ClassTag + import org.scalatest.FunSuite -import java.io.File -import org.apache.spark.rdd._ + import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 8de7a328d1cf5..14ddd6f1ec08f 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark -import network.ConnectionManagerId import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Timeouts._ import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers -import org.scalatest.time.{Span, Millis} +import org.scalatest.time.{Millis, Span} -import SparkContext._ +import org.apache.spark.SparkContext._ +import org.apache.spark.network.ConnectionManagerId import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel} - class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index fb89537258542..e0e8011278649 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ import org.scalatest.time.SpanSugar._ + import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index befdc1589f009..ac3c86778d526 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.FunSuite -import SparkContext._ +import org.apache.spark.SparkContext._ import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index a2eb9a4e84696..9be67b3c95abd 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark import java.io._ import java.util.jar.{JarEntry, JarOutputStream} -import SparkContext._ import com.google.common.io.Files import org.scalatest.FunSuite +import org.apache.spark.SparkContext._ + class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 7b82a4cdd9cfd..8ff02aef67aa0 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark -import java.io.{FileWriter, PrintWriter, File} +import java.io.{File, FileWriter} import scala.io.Source import com.google.common.io.Files -import org.scalatest.FunSuite import org.apache.hadoop.io._ -import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodec, GzipCodec} - +import org.apache.hadoop.io.compress.DefaultCodec +import org.scalatest.FunSuite -import SparkContext._ +import org.apache.spark.SparkContext._ class FileSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 8c573ac0d65e0..20232e9fbb8d0 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -22,14 +22,14 @@ import java.io.Serializable; import java.util.*; -import com.google.common.base.Optional; import scala.Tuple2; +import com.google.common.base.Optional; import com.google.common.base.Charsets; -import org.apache.hadoop.io.compress.DefaultCodec; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapreduce.Job; @@ -48,7 +48,6 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; - // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 1121e06e2e6cc..20c503d30c801 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark import java.util.concurrent.Semaphore import scala.concurrent.Await +import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.future -import scala.concurrent.ExecutionContext.Implicits.global import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.matchers.ShouldMatchers @@ -30,7 +30,6 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener} - /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action * (e.g. count) as well as multi-job action (e.g. take). We test the local and cluster schedulers diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 3ac706110e287..4b972f88a9542 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,12 +17,11 @@ package org.apache.spark -import org.scalatest.Suite -import org.scalatest.BeforeAndAfterEach -import org.scalatest.BeforeAndAfterAll - import org.jboss.netty.logging.InternalLoggerFactory import org.jboss.netty.logging.Slf4JLoggerFactory +import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach +import org.scalatest.Suite /** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 930c2523caf8c..6c1e325f6f348 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark -import org.scalatest.FunSuite +import scala.concurrent.Await import akka.actor._ +import org.scalatest.FunSuite + import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils -import scala.concurrent.Await class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { private val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 1c5d5ea4364f5..4305686d3a6d5 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -18,13 +18,12 @@ package org.apache.spark import scala.math.abs -import scala.collection.mutable.ArrayBuffer import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.SparkContext._ -import org.apache.spark.util.StatCounter import org.apache.spark.rdd.RDD +import org.apache.spark.util.StatCounter class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMethodTester { diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 2e851d892dd0d..3a0385a1b0bd9 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark import org.scalatest.FunSuite -import SparkContext._ class PipedRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index c650ef4ed58cb..0b6511a80df1d 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -17,8 +17,8 @@ package org.apache.spark -import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll +import org.scalatest.Suite /** Shares a local `SparkContext` between all tests in a suite and closes it at the end */ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index e121b162ad9e6..29d428aa7dc41 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll - class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index db717865db5d0..abea36f7c83df 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -22,10 +22,9 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass -import org.apache.spark.rdd.{RDD, SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} -import org.apache.spark.util.MutablePair +import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.serializer.KryoSerializer - +import org.apache.spark.util.MutablePair class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { test("groupByKey without compression") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 939fe518012d0..5cb49d9a7fce1 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 75d6493e338fe..b5383d553add1 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -22,9 +22,6 @@ import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import SparkContext._ /** * Holds state shared across task threads in some ThreadingSuite tests. diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index 768ca3850e7e7..42ff059e018a3 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Span, Millis} -import org.apache.spark.SparkContext._ +import org.scalatest.time.{Millis, Span} class UnpersistSuite extends FunSuite with LocalSparkContext { test("unpersist RDD") { diff --git a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala index 618b9c113b849..4f87fd8654c4a 100644 --- a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala @@ -17,16 +17,7 @@ package org.apache.spark -import scala.collection.immutable.NumericRange - import org.scalatest.FunSuite -import org.scalatest.prop.Checkers -import org.scalacheck.Arbitrary._ -import org.scalacheck.Gen -import org.scalacheck.Prop._ - -import SparkContext._ - object ZippedPartitionsSuite { def procZippedData(i: Iterator[Int], s: Iterator[String], d: Iterator[Double]) : Iterator[Int] = { diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 5bcebabc9a419..7b866f08a0e9f 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.api.python -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - import java.io.{ByteArrayOutputStream, DataOutputStream} +import org.scalatest.FunSuite + class PythonRDDSuite extends FunSuite { test("Writing large strings to the worker") { diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 6445db0063fc4..de866ed7ffed8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} -import org.apache.spark.deploy.worker.{ExecutorRunner, DriverRunner} +import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} class JsonProtocolSuite extends FunSuite { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 0c502612647a2..a2c131b0c9787 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -19,15 +19,13 @@ package org.apache.spark.deploy.worker import java.io.File -import scala.collection.JavaConversions._ - import org.mockito.Mockito._ import org.mockito.Matchers._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer import org.scalatest.FunSuite import org.apache.spark.deploy.{Command, DriverDescription} -import org.mockito.stubbing.Answer -import org.mockito.invocation.InvocationOnMock class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 4baa65659f041..3cab8e7b37934 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -21,7 +21,7 @@ import java.io.File import org.scalatest.FunSuite -import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} +import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} class ExecutorRunnerTest extends FunSuite { test("command includes appId") { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 1f1d8d138005b..0b5ed6d77034b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.deploy.worker - +import akka.actor.{ActorSystem, AddressFromURIString, Props} import akka.testkit.TestActorRef -import org.scalatest.FunSuite import akka.remote.DisassociatedEvent -import akka.actor.{ActorSystem, AddressFromURIString, Props} +import org.scalatest.FunSuite class WorkerWatcherSuite extends FunSuite { test("WorkerWatcher shuts down on valid disassociation") { diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 8d7546085f489..68a0ea36aa545 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.io import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.SparkConf class CompressionCodecSuite extends FunSuite { val conf = new SparkConf(false) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 71a2c6c498eef..c1e8b295dfe3b 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -18,8 +18,9 @@ package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.deploy.master.MasterSource + import org.apache.spark.SparkConf +import org.apache.spark.deploy.master.MasterSource class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 0d4c10db8ef33..3b833f2e41867 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -30,7 +30,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext._ import org.apache.spark.{SparkContext, SparkException, LocalSparkContext} - class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts { @transient private var sc: SparkContext = _ diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 7f50a5a47c2ff..a822bd18bfdbd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -17,14 +17,10 @@ package org.apache.spark.rdd -import scala.math.abs -import scala.collection.mutable.ArrayBuffer - import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ import org.apache.spark._ +import org.apache.spark.SparkContext._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { // Verify tests on the histogram functionality. We test with both evenly diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 3d39a31252e5e..7c7f69b261a0a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -17,11 +17,12 @@ package org.apache.spark -import org.scalatest.{ BeforeAndAfter, FunSuite } -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.JdbcRDD import java.sql._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.rdd.JdbcRDD + class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { before { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 5da538a1ddfd5..fa5c9b10fe059 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -22,12 +22,11 @@ import scala.collection.mutable.HashSet import scala.util.Random import org.scalatest.FunSuite - import com.google.common.io.Files + import org.apache.spark.SparkContext._ import org.apache.spark.{Partitioner, SharedSparkContext} - class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("groupByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index a80afdee7e769..a4381a8b974df 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.rdd import scala.collection.immutable.NumericRange -import org.scalatest.FunSuite -import org.scalatest.prop.Checkers import org.scalacheck.Arbitrary._ import org.scalacheck.Gen import org.scalacheck.Prop._ +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("one element per slice") { diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala index 53a7b7c44df1c..956c2b9cbd321 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.rdd import org.scalatest.FunSuite -import org.apache.spark.{TaskContext, Partition, SharedSparkContext} +import org.apache.spark.{Partition, SharedSparkContext, TaskContext} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index cfe96fb3f7b95..00c273df63b29 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.scalatest.FunSuite + import org.apache.spark.SharedSparkContext import org.apache.spark.util.random.RandomSampler diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 308c7cc8c3b14..60bcada55245b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -18,13 +18,15 @@ package org.apache.spark.rdd import scala.collection.mutable.HashMap +import scala.collection.parallel.mutable + import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Span, Millis} +import org.scalatest.time.{Millis, Span} + +import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ -import scala.collection.parallel.mutable -import org.apache.spark._ class RDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index e8361199421f1..d0619559bb457 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.rdd import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers import org.apache.spark.{Logging, SharedSparkContext} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala index 98ea4cb5612ec..85e929925e3b5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.scheduler -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter +import java.util.Properties -import org.apache.spark._ import scala.collection.mutable.ArrayBuffer -import java.util.Properties +import org.scalatest.FunSuite + +import org.apache.spark._ class FakeTaskSetManager( initPriority: Int, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index f0236ef1e975b..ad890b4e4d08e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.scheduler import scala.Tuple2 import scala.collection.mutable.{HashMap, Map} +import org.scalatest.{BeforeAndAfter, FunSuite} + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.scalatest.{BeforeAndAfter, FunSuite} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 29102913c719c..25fe63c2655a8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -17,11 +17,6 @@ package org.apache.spark.scheduler -import java.util.Properties -import java.util.concurrent.LinkedBlockingQueue - -import scala.collection.mutable - import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers @@ -29,7 +24,6 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD - class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { val WAIT_TIMEOUT_MILLIS = 10000 diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index e31a116a75bf3..8bb5317cd2875 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter + +import org.apache.spark.LocalSparkContext +import org.apache.spark.Partition +import org.apache.spark.SparkContext import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.Partition -import org.apache.spark.LocalSparkContext class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 4b52d9651ebe8..ac07f60e284bb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index de321c45b547c..34a7d8cefeea2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,8 +24,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import java.nio.ByteBuffer -import org.apache.spark.util.{Utils, FakeClock} +import org.apache.spark.util.FakeClock class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 3898583275557..5d4673aebe9e8 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.serializer import scala.collection.mutable import com.esotericsoftware.kryo.Kryo - import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SharedSparkContext} + +import org.apache.spark.SharedSparkContext import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 85011c6451777..9f011d9c8d132 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -20,18 +20,17 @@ package org.apache.spark.storage import java.nio.ByteBuffer import akka.actor._ - -import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter +import org.scalatest.FunSuite import org.scalatest.PrivateMethodTester import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { private val conf = new SparkConf(false) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 829f389460f3b..62f9b3cc7b2c1 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -22,9 +22,10 @@ import java.io.{File, FileWriter} import scala.collection.mutable import com.google.common.io.Files -import org.apache.spark.SparkConf import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.apache.spark.SparkConf + class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { private val testConf = new SparkConf(false) val rootDir0 = Files.createTempDir() diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index c17bbfe7d35ba..20ebb1897e6ba 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.ui -import scala.util.{Failure, Success, Try} import java.net.ServerSocket -import org.scalatest.FunSuite + +import scala.util.{Failure, Success, Try} + import org.eclipse.jetty.server.Server +import org.scalatest.FunSuite class UISuite extends FunSuite { test("jetty port increases under contention") { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 67a57a0e7f9d0..8ca863e8b3667 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite -import org.apache.spark.scheduler._ + import org.apache.spark.{LocalSparkContext, SparkContext, Success} -import org.apache.spark.scheduler.SparkListenerTaskStart import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.scheduler._ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index de4871d0433ef..439e5644e20a3 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.util -import java.io.NotSerializableException - import org.scalatest.FunSuite -import org.apache.spark.SparkContext import org.apache.spark.LocalSparkContext._ +import org.apache.spark.SparkContext class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 45867463a5132..e1446cbc90bdb 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.util +import java.util.NoSuchElementException + +import scala.collection.mutable.Buffer + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import scala.collection.mutable.Buffer -import java.util.NoSuchElementException class NextIteratorSuite extends FunSuite with ShouldMatchers { test("one iteration") { diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 11ebdc352bd74..b583a8bd46ae5 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.util -import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite import org.scalatest.PrivateMethodTester -import org.apache.spark.SparkContext class DummyClass1 {} diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 7030ba4858b9d..8f55b2372c9f1 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.util +import scala.util.Random + +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.nio.{ByteBuffer, ByteOrder} + import com.google.common.base.Charsets import com.google.common.io.Files -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} -import java.nio.{ByteBuffer, ByteOrder} -import org.scalatest.FunSuite import org.apache.commons.io.FileUtils -import scala.util.Random +import org.scalatest.FunSuite class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index f44442f1a5328..52c7288e18b69 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.util.collection +import java.util.Comparator + import scala.collection.mutable.HashSet import org.scalatest.FunSuite -import java.util.Comparator class AppendOnlyMapSuite extends FunSuite { test("initialization") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 0f1ab3d20eea4..c32183c134f9c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.collection import org.scalatest.FunSuite - class BitSetSuite extends FunSuite { test("basic set and get") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index e9b62ea70db22..b024c89d94d33 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers + import org.apache.spark.util.SizeEstimator class OpenHashMapSuite extends FunSuite with ShouldMatchers { diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 1b24f8f287cfd..ff4a98f5dcd4a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -22,7 +22,6 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.util.SizeEstimator - class OpenHashSetSuite extends FunSuite with ShouldMatchers { test("size for specialized, primitive int") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index 3b60decee9acb..e3fca173908e9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers + import org.apache.spark.util.SizeEstimator class PrimitiveKeyOpenHashMapSuite extends FunSuite with ShouldMatchers { diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index 0f4792cd3bdb3..7576c9a51f313 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.util.random -import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.mock.EasyMockSugar - import java.util.Random + import cern.jet.random.Poisson +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.mock.EasyMockSugar class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 352aa94219c2f..c51d12bfe0bc6 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util.random import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers + import org.apache.spark.util.Utils.times class XORShiftRandomSuite extends FunSuite with ShouldMatchers { From d99773d5bba674cc1434c86435b6d9b3739314c8 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 18 Feb 2014 16:29:23 -0800 Subject: [PATCH 05/32] [SPARK-1105] fix site scala version error in doc https://spark-project.atlassian.net/browse/SPARK-1105 fix site scala version error Author: CodingCat Closes #616 from CodingCat/doc_version and squashes the following commits: eafd99a [CodingCat] fix site scala version error in doc --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 98784866ce7d2..b514e414dc5a8 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -5,7 +5,7 @@ markdown: kramdown # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-incubating-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 -SCALA_VERSION: "2.10" +SCALA_VERSION: "2.10.3" MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/incubator-spark From d9bb32a790e76e35f32229082648f97170ffef07 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Feb 2014 17:46:47 -0800 Subject: [PATCH 06/32] Revert "[SPARK-1105] fix site scala version error in doc" This reverts commit d99773d5bba674cc1434c86435b6d9b3739314c8. --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index b514e414dc5a8..98784866ce7d2 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -5,7 +5,7 @@ markdown: kramdown # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-incubating-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 -SCALA_VERSION: "2.10.3" +SCALA_VERSION: "2.10" MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/incubator-spark From b61435c7ff620a05bee65607aed249541ab54b13 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 18 Feb 2014 18:30:02 -0800 Subject: [PATCH 07/32] SPARK-1106: check key name and identity file before launch a cluster I launched an EC2 cluster without providing a key name and an identity file. The error showed up after two minutes. It would be good to check those options before launch, given the fact that EC2 billing rounds up to hours. JIRA: https://spark-project.atlassian.net/browse/SPARK-1106 Author: Xiangrui Meng Closes #617 from mengxr/ec2 and squashes the following commits: 2dfb316 [Xiangrui Meng] check key name and identity file before launch a cluster --- ec2/spark_ec2.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index e88f80aa62627..b0512ca891ad6 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -218,6 +218,12 @@ def get_spark_ami(opts): # Returns a tuple of EC2 reservation objects for the master and slaves # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): + if opts.identity_file is None: + print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections." + sys.exit(1) + if opts.key_pair is None: + print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." + sys.exit(1) print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") From 7b012c93973201a1cbb4fc9a02e322152e5185a9 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 19 Feb 2014 15:54:03 -0800 Subject: [PATCH 08/32] [SPARK-1105] fix site scala version error in docs https://spark-project.atlassian.net/browse/SPARK-1105 fix site scala version error Author: CodingCat Closes #618 from CodingCat/doc_version and squashes the following commits: 39bb8aa [CodingCat] more fixes 65bedb0 [CodingCat] fix site scala version error in doc --- docs/_config.yml | 3 ++- docs/bagel-programming-guide.md | 2 +- docs/building-with-maven.md | 4 ++-- docs/index.md | 2 +- docs/quick-start.md | 4 ++-- docs/running-on-yarn.md | 16 ++++++++-------- docs/scala-programming-guide.md | 6 +++--- docs/streaming-programming-guide.md | 16 ++++++++-------- 8 files changed, 27 insertions(+), 26 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 98784866ce7d2..9e5a95fe53af6 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -5,7 +5,8 @@ markdown: kramdown # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-incubating-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 -SCALA_VERSION: "2.10" +SCALA_BINARY_VERSION: "2.10" +SCALA_VERSION: "2.10.3" MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/incubator-spark diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index cffa55ee952b0..b070d8e73a38b 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -16,7 +16,7 @@ This guide shows the programming model and features of Bagel by walking through To use Bagel in your program, add the following SBT or Maven dependency: groupId = org.apache.spark - artifactId = spark-bagel_{{site.SCALA_VERSION}} + artifactId = spark-bagel_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} # Programming Model diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 6a9a8d681742f..ded12926885b9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -17,10 +17,10 @@ You'll need to configure Maven to use more memory than usual by setting `MAVEN_O If you don't run this, you may see errors like the following: - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... [ERROR] PermGen space -> [Help 1] - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... [ERROR] Java heap space -> [Help 1] You can fix this by setting the `MAVEN_OPTS` variable as discussed before. diff --git a/docs/index.md b/docs/index.md index 7fea73024a8a0..aa9c8666e7d75 100644 --- a/docs/index.md +++ b/docs/index.md @@ -19,7 +19,7 @@ Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with sbt/sbt assembly -For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you will need to use this same version of Scala in your own program -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). +For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BINARY_VERSION}}. If you write applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). # Running the Examples and Shell diff --git a/docs/quick-start.md b/docs/quick-start.md index 153081bdaa286..13df6beea16e8 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -115,7 +115,7 @@ object SimpleApp { def main(args: Array[String]) { val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")) + List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar")) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -214,7 +214,7 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a org.apache.spark - spark-core_{{site.SCALA_VERSION}} + spark-core_{{site.SCALA_BINARY_VERSION}} {{site.SPARK_VERSION}} diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 5dadd54492dca..cd4509ede735a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -15,7 +15,7 @@ This can be built by setting the Hadoop version and `SPARK_YARN` environment var SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly The assembled JAR will be something like this: -`./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. +`./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. The build process now also supports new YARN versions (2.2.x). See below. @@ -25,7 +25,7 @@ The build process now also supports new YARN versions (2.2.x). See below. - The assembled jar can be installed into HDFS or used locally. - Your application code must be packaged into a separate JAR file. -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. # Configuration @@ -78,9 +78,9 @@ For example: $ cp conf/log4j.properties.template conf/log4j.properties # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example - $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ ./bin/spark-class org.apache.spark.deploy.yarn.Client \ - --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-standalone \ --num-workers 3 \ @@ -117,13 +117,13 @@ In order to tune worker core/number/memory etc. You need to export environment v For example: - SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ ./bin/run-example org.apache.spark.examples.SparkPi yarn-client - SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./bin/spark-shell diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 7c0f67bc99e83..cd847e07f94ab 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -17,12 +17,12 @@ This guide shows each of these features and walks through some samples. It assum # Linking with Spark -Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you'll need to use this same version of Scala in your program -- newer major versions may not work. +Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. If you write applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. To write a Spark application, you need to add a dependency on Spark. If you use SBT or Maven, Spark is available through Maven Central at: groupId = org.apache.spark - artifactId = spark-core_{{site.SCALA_VERSION}} + artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: @@ -31,7 +31,7 @@ In addition, if you wish to access an HDFS cluster, you need to add a dependency artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). +For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 924f0f4306bc2..57e88581616a2 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -275,23 +275,23 @@ To write your own Spark Streaming program, you will have to add the following de SBT or Maven project: groupId = org.apache.spark - artifactId = spark-streaming_{{site.SCALA_VERSION}} + artifactId = spark-streaming_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} For ingesting data from sources like Kafka and Flume that are not present in the Spark Streaming core API, you will have to add the corresponding -artifact `spark-streaming-xyz_{{site.SCALA_VERSION}}` to the dependencies. For example, +artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, some of the common ones are as follows. - - - - - + + + + +
SourceArtifact
Kafka spark-streaming-kafka_{{site.SCALA_VERSION}}
Flume spark-streaming-flume_{{site.SCALA_VERSION}}
Twitter spark-streaming-twitter_{{site.SCALA_VERSION}}
ZeroMQ spark-streaming-zeromq_{{site.SCALA_VERSION}}
MQTT spark-streaming-mqtt_{{site.SCALA_VERSION}}
Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
@@ -410,7 +410,7 @@ Scala and [JavaStreamingContext](api/streaming/index.html#org.apache.spark.strea Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter can be imported by adding the right dependencies as explained in an [earlier](#linking) section. To take the -case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_VERSION}}` to the +case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` to the project dependencies, you can create a DStream from Kafka as
From f9b7d64a4e7dd03be672728335cb72df4be5dbf6 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Wed, 19 Feb 2014 22:06:35 -0800 Subject: [PATCH 09/32] MLLIB-24: url of "Collaborative Filtering for Implicit Feedback Datasets" in ALS is invalid now url of "Collaborative Filtering for Implicit Feedback Datasets" is invalid now. A new url is provided. http://research.yahoo.com/files/HuKorenVolinsky-ICDM08.pdf Author: Chen Chao Closes #619 from CrazyJvm/master and squashes the following commits: a0b54e4 [Chen Chao] change url to IEEE 9e0e9f0 [Chen Chao] correct spell mistale fcfab5d [Chen Chao] wrap line to to fit within 100 chars 590d56e [Chen Chao] url error --- .../main/scala/org/apache/spark/mllib/recommendation/ALS.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index a990e0fb01c22..a566464d979f3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -81,7 +81,7 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * [[http://research.yahoo.com/pub/2433]], adapted for the blocked approach used here. + * [[http://dx.doi.org/10.1109/ICDM.2008.22]], adapted for the blocked approach used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if From 9e63f80e75bb6d9bbe6df268908c3219de6852d9 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 19 Feb 2014 23:44:53 -0800 Subject: [PATCH 10/32] MLLIB-22. Support negative implicit input in ALS I'm back with another less trivial suggestion for ALS: In ALS for implicit feedback, input values are treated as weights on squared-errors in a loss function (or rather, the weight is a simple function of the input r, like c = 1 + alpha*r). The paper on which it's based assumes that the input is positive. Indeed, if the input is negative, it will create a negative weight on squared-errors, which causes things to go haywire. The optimization will try to make the error in a cell as large possible, and the result is silently bogus. There is a good use case for negative input values though. Implicit feedback is usually collected from signals of positive interaction like a view or like or buy, but equally, can come from "not interested" signals. The natural representation is negative values. The algorithm can be extended quite simply to provide a sound interpretation of these values: negative values should encourage the factorization to come up with 0 for cells with large negative input values, just as much as positive values encourage it to come up with 1. The implications for the algorithm are simple: * the confidence function value must not be negative, and so can become 1 + alpha*|r| * the matrix P should have a value 1 where the input R is _positive_, not merely where it is non-zero. Actually, that's what the paper already says, it's just that we can't assume P = 1 when a cell in R is specified anymore, since it may be negative This in turn entails just a few lines of code change in `ALS.scala`: * `rs(i)` becomes `abs(rs(i))` * When constructing `userXy(us(i))`, it's implicitly only adding where P is 1. That had been true for any us(i) that is iterated over, before, since these are exactly the ones for which P is 1. But now P is zero where rs(i) <= 0, and should not be added I think it's a safe change because: * It doesn't change any existing behavior (unless you're using negative values, in which case results are already borked) * It's the simplest direct extension of the paper's algorithm * (I've used it to good effect in production FWIW) Tests included. I tweaked minor things en route: * `ALS.scala` javadoc writes "R = Xt*Y" when the paper and rest of code defines it as "R = X*Yt" * RMSE in the ALS tests uses a confidence-weighted mean, but the denominator is not actually sum of weights Excuse my Scala style; I'm sure it needs tweaks. Author: Sean Owen Closes #500 from srowen/ALSNegativeImplicitInput and squashes the following commits: cf902a9 [Sean Owen] Support negative implicit input in ALS 953be1c [Sean Owen] Make weighted RMSE in ALS test actually weighted; adjust comment about R = X*Yt --- .../spark/mllib/recommendation/ALS.scala | 14 ++++++-- .../mllib/recommendation/JavaALSSuite.java | 32 +++++++++++++------ .../spark/mllib/recommendation/ALSSuite.scala | 27 ++++++++++------ 3 files changed, 52 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index a566464d979f3..c668b0412c3c9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -64,7 +64,7 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * Alternating Least Squares matrix factorization. * * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, - * `X` and `Y`, i.e. `Xt * Y = R`. Typically these approximations are called 'factor' matrices. + * `X` and `Y`, i.e. `X * Yt = R`. Typically these approximations are called 'factor' matrices. * The general approach is iterative. During each iteration, one of the factor matrices is held * constant, while the other is solved for using least squares. The newly-solved factor matrix is * then held constant while solving for the other factor matrix. @@ -384,8 +384,16 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l userXtX(us(i)).addi(tempXtX) SimpleBlas.axpy(rs(i), x, userXy(us(i))) case true => - userXtX(us(i)).addi(tempXtX.mul(alpha * rs(i))) - SimpleBlas.axpy(1 + alpha * rs(i), x, userXy(us(i))) + // Extension to the original paper to handle rs(i) < 0. confidence is a function + // of |rs(i)| instead so that it is never negative: + val confidence = 1 + alpha * abs(rs(i)) + userXtX(us(i)).addi(tempXtX.mul(confidence - 1)) + // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) + // means we try to reconstruct 0. We add terms only where P = 1, so, term below + // is now only added for rs(i) > 0: + if (rs(i) > 0) { + SimpleBlas.axpy(confidence, x, userXy(us(i))) + } } } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index b40f552e0d0aa..b150334deb06c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -19,7 +19,6 @@ import java.io.Serializable; import java.util.List; -import java.lang.Math; import org.junit.After; import org.junit.Assert; @@ -46,7 +45,7 @@ public void tearDown() { System.clearProperty("spark.driver.port"); } - void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, + static void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, DoubleMatrix trueRatings, double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) { DoubleMatrix predictedU = new DoubleMatrix(users, features); List> userFeatures = model.userFeatures().toJavaRDD().collect(); @@ -84,15 +83,15 @@ void validatePrediction(MatrixFactorizationModel model, int users, int products, for (int p = 0; p < products; ++p) { double prediction = predictedRatings.get(u, p); double truePref = truePrefs.get(u, p); - double confidence = 1.0 + /* alpha = */ 1.0 * trueRatings.get(u, p); + double confidence = 1.0 + /* alpha = */ 1.0 * Math.abs(trueRatings.get(u, p)); double err = confidence * (truePref - prediction) * (truePref - prediction); sqErr += err; - denom += 1.0; + denom += confidence; } } double rmse = Math.sqrt(sqErr / denom); Assert.assertTrue(String.format("Confidence-weighted RMSE=%2.4f above threshold of %2.2f", - rmse, matchThreshold), Math.abs(rmse) < matchThreshold); + rmse, matchThreshold), rmse < matchThreshold); } } @@ -103,7 +102,7 @@ public void runALSUsingStaticMethods() { int users = 50; int products = 100; scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, false); + users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations); @@ -117,7 +116,7 @@ public void runALSUsingConstructor() { int users = 100; int products = 200; scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, false); + users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); @@ -134,7 +133,7 @@ public void runImplicitALSUsingStaticMethods() { int users = 80; int products = 160; scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true); + users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); @@ -148,7 +147,7 @@ public void runImplicitALSUsingConstructor() { int users = 100; int products = 200; scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true); + users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); @@ -158,4 +157,19 @@ public void runImplicitALSUsingConstructor() { .run(data.rdd()); validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3()); } + + @Test + public void runImplicitALSWithNegativeWeight() { + int features = 2; + int iterations = 15; + int users = 80; + int products = 160; + scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + users, products, features, 0.7, true, true); + + JavaRDD data = sc.parallelize(testData._1()); + MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); + validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3()); + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 5dcec7dc3eb9b..45e7d2db00c42 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.mllib.recommendation import scala.collection.JavaConversions._ +import scala.math.abs import scala.util.Random -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.jblas._ @@ -34,7 +34,8 @@ object ALSSuite { products: Int, features: Int, samplingRate: Double, - implicitPrefs: Boolean): (java.util.List[Rating], DoubleMatrix, DoubleMatrix) = { + implicitPrefs: Boolean, + negativeWeights: Boolean): (java.util.List[Rating], DoubleMatrix, DoubleMatrix) = { val (sampledRatings, trueRatings, truePrefs) = generateRatings(users, products, features, samplingRate, implicitPrefs) (seqAsJavaList(sampledRatings), trueRatings, truePrefs) @@ -45,7 +46,8 @@ object ALSSuite { products: Int, features: Int, samplingRate: Double, - implicitPrefs: Boolean = false): (Seq[Rating], DoubleMatrix, DoubleMatrix) = { + implicitPrefs: Boolean = false, + negativeWeights: Boolean = false): (Seq[Rating], DoubleMatrix, DoubleMatrix) = { val rand = new Random(42) // Create a random matrix with uniform values from -1 to 1 @@ -56,7 +58,9 @@ object ALSSuite { val productMatrix = randomMatrix(features, products) val (trueRatings, truePrefs) = implicitPrefs match { case true => - val raw = new DoubleMatrix(users, products, Array.fill(users * products)(rand.nextInt(10).toDouble): _*) + // Generate raw values from [0,9], or if negativeWeights, from [-2,7] + val raw = new DoubleMatrix(users, products, + Array.fill(users * products)((if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) val prefs = new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) (raw, prefs) case false => (userMatrix.mmul(productMatrix), null) @@ -107,6 +111,10 @@ class ALSSuite extends FunSuite with LocalSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, true, true) } + test("rank-2 matrices implicit negative") { + testALS(100, 200, 2, 15, 0.7, 0.4, true, false, true) + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -118,13 +126,14 @@ class ALSSuite extends FunSuite with LocalSparkContext { * @param matchThreshold max difference allowed to consider a predicted rating correct * @param implicitPrefs flag to test implicit feedback * @param bulkPredict flag to test bulk prediciton + * @param negativeWeights whether the generated data can contain negative values */ def testALS(users: Int, products: Int, features: Int, iterations: Int, samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false, - bulkPredict: Boolean = false) + bulkPredict: Boolean = false, negativeWeights: Boolean = false) { val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, - features, samplingRate, implicitPrefs) + features, samplingRate, implicitPrefs, negativeWeights) val model = implicitPrefs match { case false => ALS.train(sc.parallelize(sampledRatings), features, iterations) case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations) @@ -166,13 +175,13 @@ class ALSSuite extends FunSuite with LocalSparkContext { for (u <- 0 until users; p <- 0 until products) { val prediction = predictedRatings.get(u, p) val truePref = truePrefs.get(u, p) - val confidence = 1 + 1.0 * trueRatings.get(u, p) + val confidence = 1 + 1.0 * abs(trueRatings.get(u, p)) val err = confidence * (truePref - prediction) * (truePref - prediction) sqErr += err - denom += 1 + denom += confidence } val rmse = math.sqrt(sqErr / denom) - if (math.abs(rmse) > matchThreshold) { + if (rmse > matchThreshold) { fail("Model failed to predict RMSE: %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format( rmse, truePrefs, predictedRatings, predictedU, predictedP)) } From 3fede4831eeb7d36d4f8fa4aaa02ad0cc8b4b09e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 20 Feb 2014 16:46:13 -0800 Subject: [PATCH 11/32] Super minor: Add require for mergeCombiners in combineByKey We changed the behavior in 0.9.0 from requiring that mergeCombiners be null when mapSideCombine was false to requiring that mergeCombiners *never* be null, for external sorting. This patch adds a require() to make this behavior change explicitly messaged rather than resulting in a NPE. Author: Aaron Davidson Closes #623 from aarondav/master and squashes the following commits: 520b80c [Aaron Davidson] Super minor: Add require for mergeCombiners in combineByKey --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 15bec39659e17..39c3a4996c327 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -77,6 +77,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) partitioner: Partitioner, mapSideCombine: Boolean = true, serializerClass: String = null): RDD[(K, C)] = { + require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 if (getKeyClass().isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") From 59b1379594360636e97511982c794bcf36225e1a Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Thu, 20 Feb 2014 21:20:39 -0800 Subject: [PATCH 12/32] SPARK-1114: Allow PySpark to use existing JVM and Gateway Patch to allow PySpark to use existing JVM and Gateway. Changes to PySpark implementation of SparkConf to take existing SparkConf JVM handle. Change to PySpark SparkContext to allow subclass specific context initialization. Author: Ahir Reddy Closes #622 from ahirreddy/pyspark-existing-jvm and squashes the following commits: a86f457 [Ahir Reddy] Patch to allow PySpark to use existing JVM and Gateway. Changes to PySpark implementation of SparkConf to take existing SparkConf JVM handle. Change to PySpark SparkContext to allow subclass specific context initialization. --- python/pyspark/conf.py | 15 ++++++++++----- python/pyspark/context.py | 17 ++++++++++++----- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 3870cd8f2b097..49b68d57ab02c 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -75,7 +75,7 @@ class SparkConf(object): and can no longer be modified by the user. """ - def __init__(self, loadDefaults=True, _jvm=None): + def __init__(self, loadDefaults=True, _jvm=None, _jconf=None): """ Create a new Spark configuration. @@ -83,11 +83,16 @@ def __init__(self, loadDefaults=True, _jvm=None): properties (True by default) @param _jvm: internal parameter used to pass a handle to the Java VM; does not need to be set by users + @param _jconf: Optionally pass in an existing SparkConf handle + to use its parameters """ - from pyspark.context import SparkContext - SparkContext._ensure_initialized() - _jvm = _jvm or SparkContext._jvm - self._jconf = _jvm.SparkConf(loadDefaults) + if _jconf: + self._jconf = _jconf + else: + from pyspark.context import SparkContext + SparkContext._ensure_initialized() + _jvm = _jvm or SparkContext._jvm + self._jconf = _jvm.SparkConf(loadDefaults) def set(self, key, value): """Set a configuration property.""" diff --git a/python/pyspark/context.py b/python/pyspark/context.py index f318b5d9a73d7..93faa2e3857ed 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -51,7 +51,8 @@ class SparkContext(object): def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, - environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None): + environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, + gateway=None): """ Create a new SparkContext. At least the master and app name should be set, either through the named parameters here or through C{conf}. @@ -70,6 +71,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, unlimited batch size. @param serializer: The serializer for RDDs. @param conf: A L{SparkConf} object setting Spark properties. + @param gateway: Use an existing gateway and JVM, otherwise a new JVM + will be instatiated. >>> from pyspark.context import SparkContext @@ -80,7 +83,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ - SparkContext._ensure_initialized(self) + SparkContext._ensure_initialized(self, gateway=gateway) self.environment = environment or {} self._conf = conf or SparkConf(_jvm=self._jvm) @@ -120,7 +123,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self.environment[varName] = v # Create the Java SparkContext through Py4J - self._jsc = self._jvm.JavaSparkContext(self._conf._jconf) + self._jsc = self._initialize_context(self._conf._jconf) # Create a single Accumulator in Java that we'll send all our updates through; # they will be passed back to us through a TCP server @@ -152,11 +155,15 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self._temp_dir = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + # Initialize SparkContext in function to allow subclass specific initialization + def _initialize_context(self, jconf): + return self._jvm.JavaSparkContext(jconf) + @classmethod - def _ensure_initialized(cls, instance=None): + def _ensure_initialized(cls, instance=None, gateway=None): with SparkContext._lock: if not SparkContext._gateway: - SparkContext._gateway = launch_gateway() + SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile From 45b15e27a84527abeaa8588b0eb1ade7e831e6ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Feb 2014 11:11:55 -0800 Subject: [PATCH 13/32] SPARK-1111: URL Validation Throws Error for HDFS URL's Fixes an error where HDFS URL's cause an exception. Should be merged into master and 0.9. Author: Patrick Wendell Closes #625 from pwendell/url-validation and squashes the following commits: d14bfe3 [Patrick Wendell] SPARK-1111: URL Validation Throws Error for HDFS URL's --- .../apache/spark/deploy/ClientArguments.scala | 17 +++++----- .../org/apache/spark/deploy/ClientSuite.scala | 34 +++++++++++++++++++ 2 files changed, 42 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 3db970ca73b92..00f5cd54ad650 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import java.net.URL - import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -71,13 +69,10 @@ private[spark] class ClientArguments(args: Array[String]) { case "launch" :: _master :: _jarUrl :: _mainClass :: tail => cmd = "launch" - try { - new URL(_jarUrl) - } catch { - case e: Exception => - println(s"Jar url '${_jarUrl}' is not a valid URL.") - println(s"Jar must be in URL format (e.g. hdfs://XX, file://XX)") - printUsageAndExit(-1) + if (!ClientArguments.isValidJarUrl(_jarUrl)) { + println(s"Jar url '${_jarUrl}' is not in valid format.") + println(s"Must be a jar file path in URL format (e.g. hdfs://XX.jar, file://XX.jar)") + printUsageAndExit(-1) } jarUrl = _jarUrl @@ -115,3 +110,7 @@ private[spark] class ClientArguments(args: Array[String]) { System.exit(exitCode) } } + +object ClientArguments { + def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar") +} diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala new file mode 100644 index 0000000000000..d6b93f5fedd3b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers + +class ClientSuite extends FunSuite with ShouldMatchers { + test("correctly validates driver jar URL's") { + ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true) + ClientArguments.isValidJarUrl("file://some/path/to/a/jarFile.jar") should be (true) + ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo.jar") should be (true) + + ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false) + ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false) + ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false) + } + +} From c8a4c9b1f6005815f5a4a331970624d1706b6b13 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 21 Feb 2014 12:46:12 -0800 Subject: [PATCH 14/32] MLLIB-25: Implicit ALS runs out of memory for moderately large numbers of features There's a step in implicit ALS where the matrix `Yt * Y` is computed. It's computed as the sum of matrices; an f x f matrix is created for each of n user/item rows in a partition. In `ALS.scala:214`: ``` factors.flatMapValues{ case factorArray => factorArray.map{ vector => val x = new DoubleMatrix(vector) x.mmul(x.transpose()) } }.reduceByKeyLocally((a, b) => a.addi(b)) .values .reduce((a, b) => a.addi(b)) ``` Completely correct, but there's a subtle but quite large memory problem here. map() is going to create all of these matrices in memory at once, when they don't need to ever all exist at the same time. For example, if a partition has n = 100000 rows, and f = 200, then this intermediate product requires 32GB of heap. The computation will never work unless you can cough up workers with (more than) that much heap. Fortunately there's a trivial change that fixes it; just add `.view` in there. Author: Sean Owen Closes #629 from srowen/ALSMatrixAllocationOptimization and squashes the following commits: 062cda9 [Sean Owen] Update style per review comments e9a5d63 [Sean Owen] Avoid unnecessary out of memory situation by not simultaneously allocating lots of matrices --- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index c668b0412c3c9..8958040e36640 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -211,8 +211,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { if (implicitPrefs) { Option( - factors.flatMapValues{ case factorArray => - factorArray.map{ vector => + factors.flatMapValues { case factorArray => + factorArray.view.map { vector => val x = new DoubleMatrix(vector) x.mmul(x.transpose()) } From fefd22f4c3e95d904cb6f4f3fd88b89050907ae9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 21 Feb 2014 20:05:39 -0800 Subject: [PATCH 15/32] [SPARK-1113] External spilling - fix Int.MaxValue hash code collision bug The original poster of this bug is @guojc, who opened a PR that preceded this one at https://github.com/apache/incubator-spark/pull/612. ExternalAppendOnlyMap uses key hash code to order the buffer streams from which spilled files are read back into memory. When a buffer stream is empty, the default hash code for that stream is equal to Int.MaxValue. This is, however, a perfectly legitimate candidate for a key hash code. When reading from a spilled map containing such a key, a hash collision may occur, in which case we attempt to read from an empty stream and throw NoSuchElementException. The fix is to maintain the invariant that empty buffer streams are never added back to the merge queue to be considered. This guarantees that we never read from an empty buffer stream, ever again. This PR also includes two new tests for hash collisions. Author: Andrew Or Closes #624 from andrewor14/spilling-bug and squashes the following commits: 9e7263d [Andrew Or] Slightly optimize next() 2037ae2 [Andrew Or] Move a few comments around... cf95942 [Andrew Or] Remove default value of Int.MaxValue for minKeyHash c11f03b [Andrew Or] Fix Int.MaxValue hash collision bug in ExternalAppendOnlyMap 21c1a39 [Andrew Or] Add hash collision tests to ExternalAppendOnlyMapSuite --- .../collection/ExternalAppendOnlyMap.scala | 49 +++++----- .../ExternalAppendOnlyMapSuite.scala | 91 ++++++++++++++++--- 2 files changed, 102 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 856d092ab3463..ed74a31f05bae 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -148,7 +148,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } /** - * Sort the existing contents of the in-memory map and spill them to a temporary file on disk + * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ private def spill(mapSize: Long) { spillCount += 1 @@ -223,7 +223,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( */ private class ExternalIterator extends Iterator[(K, C)] { - // A fixed-size queue that maintains a buffer for each stream we are currently merging + // A queue that maintains a buffer for each stream we are currently merging + // This queue maintains the invariant that it only contains non-empty buffers private val mergeHeap = new mutable.PriorityQueue[StreamBuffer] // Input streams are derived both from the in-memory map and spilled maps on disk @@ -233,7 +234,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( inputStreams.foreach { it => val kcPairs = getMorePairs(it) - mergeHeap.enqueue(StreamBuffer(it, kcPairs)) + if (kcPairs.length > 0) { + mergeHeap.enqueue(new StreamBuffer(it, kcPairs)) + } } /** @@ -258,11 +261,11 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * If the given buffer contains a value for the given key, merge that value into - * baseCombiner and remove the corresponding (K, C) pair from the buffer + * baseCombiner and remove the corresponding (K, C) pair from the buffer. */ private def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { var i = 0 - while (i < buffer.pairs.size) { + while (i < buffer.pairs.length) { val (k, c) = buffer.pairs(i) if (k == key) { buffer.pairs.remove(i) @@ -274,40 +277,41 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } /** - * Return true if there exists an input stream that still has unvisited pairs + * Return true if there exists an input stream that still has unvisited pairs. */ - override def hasNext: Boolean = mergeHeap.exists(!_.pairs.isEmpty) + override def hasNext: Boolean = mergeHeap.length > 0 /** * Select a key with the minimum hash, then combine all values with the same key from all * input streams. */ override def next(): (K, C) = { + if (mergeHeap.length == 0) { + throw new NoSuchElementException + } // Select a key from the StreamBuffer that holds the lowest key hash val minBuffer = mergeHeap.dequeue() val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) - if (minPairs.length == 0) { - // Should only happen when no other stream buffers have any pairs left - throw new NoSuchElementException - } var (minKey, minCombiner) = minPairs.remove(0) assert(minKey.hashCode() == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), // merge in the corresponding value (if any) from that stream val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) - while (!mergeHeap.isEmpty && mergeHeap.head.minKeyHash == minHash) { + while (mergeHeap.length > 0 && mergeHeap.head.minKeyHash == minHash) { val newBuffer = mergeHeap.dequeue() minCombiner = mergeIfKeyExists(minKey, minCombiner, newBuffer) mergedBuffers += newBuffer } - // Repopulate each visited stream buffer and add it back to the merge heap + // Repopulate each visited stream buffer and add it back to the queue if it is non-empty mergedBuffers.foreach { buffer => - if (buffer.pairs.length == 0) { + if (buffer.isEmpty) { buffer.pairs ++= getMorePairs(buffer.iterator) } - mergeHeap.enqueue(buffer) + if (!buffer.isEmpty) { + mergeHeap.enqueue(buffer) + } } (minKey, minCombiner) @@ -323,13 +327,12 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { - def minKeyHash: Int = { - if (pairs.length > 0){ - // pairs are already sorted by key hash - pairs(0)._1.hashCode() - } else { - Int.MaxValue - } + def isEmpty = pairs.length == 0 + + // Invalid if there are no more pairs in this stream + def minKeyHash = { + assert(pairs.length > 0) + pairs.head._1.hashCode() } override def compareTo(other: StreamBuffer): Int = { @@ -356,7 +359,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var objectsRead = 0 /** - * Construct a stream that reads only from the next batch + * Construct a stream that reads only from the next batch. */ private def nextBatchStream(): InputStream = { if (batchSizes.length > 0) { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index bb4dc0fcd31a3..fce1184d46364 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -19,21 +19,16 @@ package org.apache.spark.util.collection import scala.collection.mutable.ArrayBuffer -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.SparkContext._ -class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { +class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { - private val createCombiner: (Int => ArrayBuffer[Int]) = i => ArrayBuffer[Int](i) - private val mergeValue: (ArrayBuffer[Int], Int) => ArrayBuffer[Int] = (buffer, i) => { - buffer += i - } - private val mergeCombiners: (ArrayBuffer[Int], ArrayBuffer[Int]) => ArrayBuffer[Int] = - (buf1, buf2) => { - buf1 ++= buf2 - } + private def createCombiner(i: Int) = ArrayBuffer[Int](i) + private def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i + private def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 test("simple insert") { val conf = new SparkConf(false) @@ -203,13 +198,13 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } test("spilling") { - // TODO: Use SparkConf (which currently throws connection reset exception) - System.setProperty("spark.shuffle.memoryFraction", "0.001") - sc = new SparkContext("local-cluster[1,1,512]", "test") + val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) // reduceByKey - should spill ~8 times val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) - val resultA = rddA.reduceByKey(math.max(_, _)).collect() + val resultA = rddA.reduceByKey(math.max).collect() assert(resultA.length == 50000) resultA.foreach { case(k, v) => k match { @@ -252,7 +247,73 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local case _ => } } + } + + test("spilling with hash collisions") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: String) = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = + buffer1 ++= buffer2 + + val map = new ExternalAppendOnlyMap[String, String, ArrayBuffer[String]]( + createCombiner, mergeValue, mergeCombiners) + + val collisionPairs = Seq( + ("Aa", "BB"), // 2112 + ("to", "v1"), // 3707 + ("variants", "gelato"), // -1249574770 + ("Teheran", "Siblings"), // 231609873 + ("misused", "horsemints"), // 1069518484 + ("isohel", "epistolaries"), // -1179291542 + ("righto", "buzzards"), // -931102253 + ("hierarch", "crinolines"), // -1732884796 + ("inwork", "hypercatalexes"), // -1183663690 + ("wainages", "presentencing"), // 240183619 + ("trichothecenes", "locular"), // 339006536 + ("pomatoes", "eructation") // 568647356 + ) + + (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } + collisionPairs.foreach { case (w1, w2) => + map.insert(w1, w2) + map.insert(w2, w1) + } + + // A map of collision pairs in both directions + val collisionPairsMap = (collisionPairs ++ collisionPairs.map(_.swap)).toMap + + // Avoid map.size or map.iterator.length because this destructively sorts the underlying map + var count = 0 + + val it = map.iterator + while (it.hasNext) { + val kv = it.next() + val expectedValue = ArrayBuffer[String](collisionPairsMap.getOrElse(kv._1, kv._1)) + assert(kv._2.equals(expectedValue)) + count += 1 + } + assert(count == 100000 + collisionPairs.size * 2) + } + + test("spilling with hash collisions using the Int.MaxValue key") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - System.clearProperty("spark.shuffle.memoryFraction") + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + + (1 to 100000).foreach { i => map.insert(i, i) } + map.insert(Int.MaxValue, Int.MaxValue) + + val it = map.iterator + while (it.hasNext) { + // Should not throw NoSuchElementException + it.next() + } } } From aaec7d4a80ed370847671e9e29ce2e92f1cff2c7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 21 Feb 2014 22:44:45 -0800 Subject: [PATCH 16/32] SPARK-1117: update accumulator docs The current doc hints spark doesn't support accumulators of type `Long`, which is wrong. JIRA: https://spark-project.atlassian.net/browse/SPARK-1117 Author: Xiangrui Meng Closes #631 from mengxr/acc and squashes the following commits: 45ecd25 [Xiangrui Meng] update accumulator docs --- core/src/main/scala/org/apache/spark/Accumulators.scala | 4 ++-- docs/scala-programming-guide.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 73dd471ab1d52..d5f3e3f6ec496 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -189,8 +189,8 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser * A simpler value of [[Accumulable]] where the result type being accumulated is the same * as the types of elements being merged, i.e. variables that are only "added" to through an * associative operation and can therefore be efficiently supported in parallel. They can be used - * to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of type - * `Int` and `Double`, and programmers can add support for new types. + * to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric + * value types, and programmers can add support for new types. * * An accumulator is created from an initial value `v` by calling [[SparkContext#accumulator]]. * Tasks running on the cluster can then add to it using the [[Accumulable#+=]] operator. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index cd847e07f94ab..506d3faa767f3 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -344,7 +344,7 @@ After the broadcast variable is created, it should be used instead of the value ## Accumulators -Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of type Int and Double, and programmers can add support for new types. +Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable collections, and programmers can add support for new types. An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method. From 3ff077d489af99ad36c9d2389e2afab6465648d4 Mon Sep 17 00:00:00 2001 From: jyotiska Date: Sat, 22 Feb 2014 10:09:50 -0800 Subject: [PATCH 17/32] Fixed minor typo in worker.py Fixed minor typo in worker.py Author: jyotiska Closes #630 from jyotiska/pyspark_code and squashes the following commits: ee44201 [jyotiska] typo fixed in worker.py --- python/pyspark/worker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4be4063dcf602..158646352039f 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -76,7 +76,7 @@ def main(infile, outfile): iterator = deserializer.load_stream(infile) serializer.dump_stream(func(split_index, iterator), outfile) except Exception as e: - # Write the error to stderr in addition to trying to passi t back to + # Write the error to stderr in addition to trying to pass it back to # Java, in case it happened while serializing a record print >> sys.stderr, "PySpark worker failed with exception:" print >> sys.stderr, traceback.format_exc() From 722199fab072b4c19a82031c52e5d44f300bd2ea Mon Sep 17 00:00:00 2001 From: jyotiska Date: Sat, 22 Feb 2014 15:10:31 -0800 Subject: [PATCH 18/32] doctest updated for mapValues, flatMapValues in rdd.py Updated doctests for mapValues and flatMapValues in rdd.py Author: jyotiska Closes #621 from jyotiska/python_spark and squashes the following commits: 716f7cd [jyotiska] doctest updated for mapValues, flatMapValues in rdd.py --- python/pyspark/rdd.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 90f93a19264bd..1330e6146800c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -946,6 +946,11 @@ def flatMapValues(self, f): Pass each value in the key-value pair RDD through a flatMap function without changing the keys; this also retains the original RDD's partitioning. + + >>> x = sc.parallelize([("a", ["x", "y", "z"]), ("b", ["p", "r"])]) + >>> def f(x): return x + >>> x.flatMapValues(f).collect() + [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')] """ flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) return self.flatMap(flat_map_fn, preservesPartitioning=True) @@ -955,6 +960,11 @@ def mapValues(self, f): Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning. + + >>> x = sc.parallelize([("a", ["apple", "banana", "lemon"]), ("b", ["grapes"])]) + >>> def f(x): return len(x) + >>> x.mapValues(f).collect() + [('a', 3), ('b', 1)] """ map_values_fn = lambda (k, v): (k, f(v)) return self.map(map_values_fn, preservesPartitioning=True) From 1aa4f8af7220bc311196ef0eef0a4814cd2757d3 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sat, 22 Feb 2014 15:39:25 -0800 Subject: [PATCH 19/32] [SPARK-1055] fix the SCALA_VERSION and SPARK_VERSION in docker file As reported in https://spark-project.atlassian.net/browse/SPARK-1055 "The used Spark version in the .../base/Dockerfile is stale on 0.8.1 and should be updated to 0.9.x to match the release." Author: CodingCat Author: Nan Zhu Closes #634 from CodingCat/SPARK-1055 and squashes the following commits: cb7330e [Nan Zhu] Update Dockerfile adf8259 [CodingCat] fix the SCALA_VERSION and SPARK_VERSION in docker file --- docker/spark-test/base/Dockerfile | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile index 60962776dda57..e543db6143e4d 100644 --- a/docker/spark-test/base/Dockerfile +++ b/docker/spark-test/base/Dockerfile @@ -25,8 +25,7 @@ RUN apt-get update # install a few other useful packages plus Open Jdk 7 RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server -ENV SCALA_VERSION 2.9.3 -ENV SPARK_VERSION 0.8.1 +ENV SCALA_VERSION 2.10.3 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark From 29ac7ea52fbb0c6531e14305e2fb1ccba9678f7e Mon Sep 17 00:00:00 2001 From: Punya Biswal Date: Sat, 22 Feb 2014 17:53:48 -0800 Subject: [PATCH 20/32] Migrate Java code to Scala or move it to src/main/java These classes can't be migrated: StorageLevels: impossible to create static fields in Scala JavaSparkContextVarargsWorkaround: incompatible varargs JavaAPISuite: should test Java APIs in pure Java (for sanity) Author: Punya Biswal Closes #605 from punya/move-java-sources and squashes the following commits: 25b00b2 [Punya Biswal] Remove redundant type param; reformat 853da46 [Punya Biswal] Use factory method rather than constructor e5d53d9 [Punya Biswal] Migrate Java code to Scala or move it to src/main/java --- .../JavaSparkContextVarargsWorkaround.java | 0 .../apache/spark/api/java/StorageLevels.java | 22 +++++++++--------- .../{SparkFiles.java => SparkFiles.scala} | 19 +++++++-------- ...ction.java => DoubleFlatMapFunction.scala} | 10 ++++---- ...ubleFunction.java => DoubleFunction.scala} | 7 +++--- .../{Function.java => Function.scala} | 14 ++++------- .../{Function2.java => Function2.scala} | 16 ++++--------- .../{Function3.java => Function3.scala} | 17 ++++---------- ...unction.java => PairFlatMapFunction.scala} | 23 +++++++------------ .../{PairFunction.java => PairFunction.scala} | 20 +++++----------- .../org/apache/spark/JavaAPISuite.java | 0 11 files changed, 56 insertions(+), 92 deletions(-) rename core/src/main/{scala => java}/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java (100%) rename core/src/main/{scala => java}/org/apache/spark/api/java/StorageLevels.java (58%) rename core/src/main/scala/org/apache/spark/{SparkFiles.java => SparkFiles.scala} (77%) rename core/src/main/scala/org/apache/spark/api/java/function/{DoubleFlatMapFunction.java => DoubleFlatMapFunction.scala} (80%) rename core/src/main/scala/org/apache/spark/api/java/function/{DoubleFunction.java => DoubleFunction.scala} (85%) rename core/src/main/scala/org/apache/spark/api/java/function/{Function.java => Function.scala} (76%) rename core/src/main/scala/org/apache/spark/api/java/function/{Function2.java => Function2.scala} (72%) rename core/src/main/scala/org/apache/spark/api/java/function/{Function3.java => Function3.scala} (70%) rename core/src/main/scala/org/apache/spark/api/java/function/{PairFlatMapFunction.java => PairFlatMapFunction.scala} (69%) rename core/src/main/scala/org/apache/spark/api/java/function/{PairFunction.java => PairFunction.scala} (68%) rename core/src/test/{scala => java}/org/apache/spark/JavaAPISuite.java (100%) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java similarity index 100% rename from core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java rename to core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java diff --git a/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java similarity index 58% rename from core/src/main/scala/org/apache/spark/api/java/StorageLevels.java rename to core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 0744269773f8d..9f13b39909481 100644 --- a/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -23,17 +23,17 @@ * Expose some commonly useful storage level constants. */ public class StorageLevels { - public static final StorageLevel NONE = new StorageLevel(false, false, false, 1); - public static final StorageLevel DISK_ONLY = new StorageLevel(true, false, false, 1); - public static final StorageLevel DISK_ONLY_2 = new StorageLevel(true, false, false, 2); - public static final StorageLevel MEMORY_ONLY = new StorageLevel(false, true, true, 1); - public static final StorageLevel MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2); - public static final StorageLevel MEMORY_ONLY_SER = new StorageLevel(false, true, false, 1); - public static final StorageLevel MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2); - public static final StorageLevel MEMORY_AND_DISK = new StorageLevel(true, true, true, 1); - public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2); - public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, 1); - public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2); + public static final StorageLevel NONE = create(false, false, false, 1); + public static final StorageLevel DISK_ONLY = create(true, false, false, 1); + public static final StorageLevel DISK_ONLY_2 = create(true, false, false, 2); + public static final StorageLevel MEMORY_ONLY = create(false, true, true, 1); + public static final StorageLevel MEMORY_ONLY_2 = create(false, true, true, 2); + public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, 1); + public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, 2); + public static final StorageLevel MEMORY_AND_DISK = create(true, true, true, 1); + public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, true, 2); + public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, 1); + public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, 2); /** * Create a new StorageLevel object. diff --git a/core/src/main/scala/org/apache/spark/SparkFiles.java b/core/src/main/scala/org/apache/spark/SparkFiles.scala similarity index 77% rename from core/src/main/scala/org/apache/spark/SparkFiles.java rename to core/src/main/scala/org/apache/spark/SparkFiles.scala index af9cf85e372bf..e85b89fd014ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkFiles.java +++ b/core/src/main/scala/org/apache/spark/SparkFiles.scala @@ -15,28 +15,25 @@ * limitations under the License. */ -package org.apache.spark; +package org.apache.spark -import java.io.File; +import java.io.File /** * Resolves paths to files added through `SparkContext.addFile()`. */ -public class SparkFiles { - - private SparkFiles() {} +object SparkFiles { /** * Get the absolute path of a file added through `SparkContext.addFile()`. */ - public static String get(String filename) { - return new File(getRootDirectory(), filename).getAbsolutePath(); - } + def get(filename: String): String = + new File(getRootDirectory(), filename).getAbsolutePath() /** * Get the root directory that contains files added through `SparkContext.addFile()`. */ - public static String getRootDirectory() { - return SparkEnv.get().sparkFilesDir(); - } + def getRootDirectory(): String = + SparkEnv.get.sparkFilesDir + } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala similarity index 80% rename from core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala index 30e6a524742c4..7500a8943634b 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; +import java.lang.{Double => JDouble, Iterable => JIterable} /** * A function that returns zero or more records of type Double from each input record. */ // DoubleFlatMapFunction does not extend FlatMapFunction because flatMap is // overloaded for both FlatMapFunction and DoubleFlatMapFunction. -public abstract class DoubleFlatMapFunction extends WrappedFunction1> - implements Serializable { - // Intentionally left blank +abstract class DoubleFlatMapFunction[T] extends WrappedFunction1[T, JIterable[JDouble]] + with Serializable { + // Intentionally left blank } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala similarity index 85% rename from core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala index 490da255bc912..2cdf2e92c3daa 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; +import java.lang.{Double => JDouble} /** * A function that returns Doubles, and can be used to construct DoubleRDDs. */ // DoubleFunction does not extend Function because some UDF functions, like map, // are overloaded for both Function and DoubleFunction. -public abstract class DoubleFunction extends WrappedFunction1 - implements Serializable { +abstract class DoubleFunction[T] extends WrappedFunction1[T, JDouble] with Serializable { // Intentionally left blank } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.scala similarity index 76% rename from core/src/main/scala/org/apache/spark/api/java/function/Function.java rename to core/src/main/scala/org/apache/spark/api/java/function/Function.scala index e0fcd460c8404..a5e1701f7718f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.scala @@ -15,21 +15,17 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; - -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; +import scala.reflect.ClassTag +import org.apache.spark.api.java.JavaSparkContext /** * Base class for functions whose return types do not create special RDDs. PairFunction and * DoubleFunction are handled separately, to allow PairRDDs and DoubleRDDs to be constructed * when mapping RDDs of other types. */ -public abstract class Function extends WrappedFunction1 implements Serializable { - public ClassTag returnType() { - return ClassTag$.MODULE$.apply(Object.class); - } +abstract class Function[T, R] extends WrappedFunction1[T, R] with Serializable { + def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.scala similarity index 72% rename from core/src/main/scala/org/apache/spark/api/java/function/Function2.java rename to core/src/main/scala/org/apache/spark/api/java/function/Function2.scala index 16d7379462e15..fa3616cbcb4d2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.scala @@ -15,21 +15,15 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; - -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; +import scala.reflect.ClassTag +import org.apache.spark.api.java.JavaSparkContext /** * A two-argument function that takes arguments of type T1 and T2 and returns an R. */ -public abstract class Function2 extends WrappedFunction2 - implements Serializable { - - public ClassTag returnType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } +abstract class Function2[T1, T2, R] extends WrappedFunction2[T1, T2, R] with Serializable { + def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java b/core/src/main/scala/org/apache/spark/api/java/function/Function3.scala similarity index 70% rename from core/src/main/scala/org/apache/spark/api/java/function/Function3.java rename to core/src/main/scala/org/apache/spark/api/java/function/Function3.scala index 096eb71f95cd0..45152891e9272 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.scala @@ -15,21 +15,14 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; - -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; +import org.apache.spark.api.java.JavaSparkContext +import scala.reflect.ClassTag /** * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R. */ -public abstract class Function3 extends WrappedFunction3 - implements Serializable { - - public ClassTag returnType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } +abstract class Function3[T1, T2, T3, R] extends WrappedFunction3[T1, T2, T3, R] with Serializable { + def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag } - diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala similarity index 69% rename from core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala index c72b98c28a0cb..8467bbb892ab0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; - -import scala.Tuple2; -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; +import java.lang.{Iterable => JIterable} +import org.apache.spark.api.java.JavaSparkContext +import scala.reflect.ClassTag /** * A function that returns zero or more key-value pair records from each input record. The @@ -29,15 +27,10 @@ */ // PairFlatMapFunction does not extend FlatMapFunction because flatMap is // overloaded for both FlatMapFunction and PairFlatMapFunction. -public abstract class PairFlatMapFunction - extends WrappedFunction1>> - implements Serializable { +abstract class PairFlatMapFunction[T, K, V] extends WrappedFunction1[T, JIterable[(K, V)]] + with Serializable { - public ClassTag keyType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } + def keyType(): ClassTag[K] = JavaSparkContext.fakeClassTag - public ClassTag valueType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } + def valueType(): ClassTag[V] = JavaSparkContext.fakeClassTag } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala similarity index 68% rename from core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala index 84b9136d98aa2..d0ba0b6307ee9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala @@ -15,27 +15,19 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import java.io.Serializable; - -import scala.Tuple2; -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; +import scala.reflect.ClassTag +import org.apache.spark.api.java.JavaSparkContext /** * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. */ // PairFunction does not extend Function because some UDF functions, like map, // are overloaded for both Function and PairFunction. -public abstract class PairFunction extends WrappedFunction1> - implements Serializable { +abstract class PairFunction[T, K, V] extends WrappedFunction1[T, (K, V)] with Serializable { - public ClassTag keyType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } + def keyType(): ClassTag[K] = JavaSparkContext.fakeClassTag - public ClassTag valueType() { - return (ClassTag) ClassTag$.MODULE$.apply(Object.class); - } + def valueType(): ClassTag[V] = JavaSparkContext.fakeClassTag } diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java similarity index 100% rename from core/src/test/scala/org/apache/spark/JavaAPISuite.java rename to core/src/test/java/org/apache/spark/JavaAPISuite.java From 437b62fcb03cc84c6581eb667ddbddd817f7f9f0 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sat, 22 Feb 2014 20:21:15 -0800 Subject: [PATCH 21/32] [SPARK-1041] remove dead code in start script, remind user to set that in spark-env.sh the lines in start-master.sh and start-slave.sh no longer work in ec2, the host name has changed, e.g. ubuntu@ip-172-31-36-93:~$ hostname ip-172-31-36-93 also, the URL to fetch public DNS name also changed, e.g. ubuntu@ip-172-31-36-93:~$ wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname ubuntu@ip-172-31-36-93:~$ (returns nothing) since we have spark-ec2 project, we don't need to have such ec2-specific lines here, instead, user only need to set in spark-env.sh Author: CodingCat Closes #588 from CodingCat/deadcode_in_sbin and squashes the following commits: e4236e0 [CodingCat] remove dead code in start script, remind user set that in spark-env.sh --- conf/spark-env.sh.template | 1 + sbin/start-master.sh | 9 --------- sbin/start-slave.sh | 9 --------- 3 files changed, 1 insertion(+), 18 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index e2071e2ade8cd..6432a566089be 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -19,3 +19,4 @@ # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes +# - SPARK_PUBLIC_DNS, to set the public dns name of the master diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 3dcf7cc3483a7..ec3dfdb4197ec 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -40,13 +40,4 @@ if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then SPARK_MASTER_WEBUI_PORT=8080 fi -# Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves -if [ "$SPARK_PUBLIC_DNS" = "" ]; then - # If we appear to be running on EC2, use the public address by default: - # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname - if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then - export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` - fi -fi - "$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 524be38c62968..b563400dc24f3 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -23,13 +23,4 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` -# Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI -if [ "$SPARK_PUBLIC_DNS" = "" ]; then - # If we appear to be running on EC2, use the public address by default: - # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname - if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then - export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` - fi -fi - "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" From c0ef3afa82c1eaf58ff5efec961540a74b639fd9 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 23 Feb 2014 11:40:55 -0800 Subject: [PATCH 22/32] SPARK-1071: Tidy logging strategy and use of log4j MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Prompted by a recent thread on the mailing list, I tried and failed to see if Spark can be made independent of log4j. There are a few cases where control of the underlying logging is pretty useful, and to do that, you have to bind to a specific logger. Instead I propose some tidying that leaves Spark's use of log4j, but gets rid of warnings and should still enable downstream users to switch. The idea is to pipe everything (except log4j) through SLF4J, and have Spark use SLF4J directly when logging, and where Spark needs to output info (REPL and tests), bind from SLF4J to log4j. This leaves the same behavior in Spark. It means that downstream users who want to use something except log4j should: - Exclude dependencies on log4j, slf4j-log4j12 from Spark - Include dependency on log4j-over-slf4j - Include dependency on another logger X, and another slf4j-X - Recreate any log config that Spark does, that is needed, in the other logger's config That sounds about right. Here are the key changes: - Include the jcl-over-slf4j shim everywhere by depending on it in core. - Exclude dependencies on commons-logging from third-party libraries. - Include the jul-to-slf4j shim everywhere by depending on it in core. - Exclude slf4j-* dependencies from third-party libraries to prevent collision or warnings - Added missing slf4j-log4j12 binding to GraphX, Bagel module tests And minor/incidental changes: - Update to SLF4J 1.7.5, which happily matches Hadoop 2’s version and is a recommended update over 1.7.2 - (Remove a duplicate HBase dependency declaration in SparkBuild.scala) - (Remove a duplicate mockito dependency declaration that was causing warnings and bugging me) Author: Sean Owen Closes #570 from srowen/SPARK-1071 and squashes the following commits: 52eac9f [Sean Owen] Add slf4j-over-log4j12 dependency to core (non-test) and remove it from things that depend on core. 77a7fa9 [Sean Owen] SPARK-1071: Tidy logging strategy and use of log4j --- core/pom.xml | 31 ++++++++++++++++++++++--------- examples/pom.xml | 8 ++++---- external/flume/pom.xml | 5 ----- external/kafka/pom.xml | 9 ++++----- external/mqtt/pom.xml | 5 ----- external/twitter/pom.xml | 5 ----- external/zeromq/pom.xml | 5 ----- pom.xml | 28 ++++++++++++++++------------ project/SparkBuild.scala | 21 +++++++++++---------- repl/pom.xml | 4 ---- streaming/pom.xml | 5 ----- 11 files changed, 57 insertions(+), 69 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index dbcde0f434f94..5576b0c3b4795 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -39,6 +39,12 @@ net.java.dev.jets3t jets3t + + + commons-logging + commons-logging + + org.apache.avro @@ -68,6 +74,22 @@ org.slf4j slf4j-api + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + com.ning compress-lzf @@ -127,10 +149,6 @@ io.netty netty-all - - log4j - log4j - com.clearspring.analytics stream @@ -190,11 +208,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/examples/pom.xml b/examples/pom.xml index 874bbd803fe5f..12a11821a4947 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -115,6 +115,10 @@ org.jboss.netty netty + + commons-logging + commons-logging + @@ -161,10 +165,6 @@ jline jline - - log4j - log4j - org.apache.cassandra.deps avro diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cdb9bef8e2dc6..a0e8b84514ef6 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -74,11 +74,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 630bbed5eb006..fb37cd79884c8 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -61,6 +61,10 @@ net.sf.jopt-simple jopt-simple + + org.slf4j + slf4j-simple + @@ -78,11 +82,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index eff3e7809b786..cfa1870e982fe 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -89,11 +89,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index cde495cac4f97..077f88dc59bab 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -70,11 +70,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 04f9fbfd5bf01..4c68294b7b5af 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -70,11 +70,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - target/scala-${scala.binary.version}/classes diff --git a/pom.xml b/pom.xml index f94685652ad61..3a530685b8e5a 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 0.13.0 org.spark-project.akka 2.2.3-shaded-protobuf - 1.7.2 + 1.7.5 1.2.17 1.0.4 2.4.1 @@ -185,6 +185,17 @@ jul-to-slf4j ${slf4j.version} + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + + + log4j + log4j + ${log4j.version} + com.ning compress-lzf @@ -346,23 +357,12 @@ scala-library ${scala.version} - - log4j - log4j - ${log4j.version} - org.scalatest scalatest_${scala.binary.version} 1.9.1 test - - org.mockito - mockito-all - test - 1.8.5 - commons-io commons-io @@ -424,6 +424,10 @@ org.sonatype.sisu.inject * + + commons-logging + commons-logging + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 74bad66cfd018..f0d2e741484f9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -86,7 +86,7 @@ object SparkBuild extends Build { case None => DEFAULT_YARN case Some(v) => v.toBoolean } - lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" + lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" // Conditionally include the yarn sub-project lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) @@ -236,13 +236,15 @@ object SparkBuild extends Build { publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings - val slf4jVersion = "1.7.2" + val slf4jVersion = "1.7.5" val excludeCglib = ExclusionRule(organization = "org.sonatype.sisu.inject") val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") val excludeAsm = ExclusionRule(organization = "asm") val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") + val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") + val excludeSLF4J = ExclusionRule(organization = "org.slf4j") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -257,6 +259,8 @@ object SparkBuild extends Build { "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "org.slf4j" % "jul-to-slf4j" % slf4jVersion, + "org.slf4j" % "jcl-over-slf4j" % slf4jVersion, "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", @@ -268,9 +272,9 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", - "net.java.dev.jets3t" % "jets3t" % "0.7.1", + "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), "org.apache.avro" % "avro" % "1.7.4", "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty), @@ -300,18 +304,15 @@ object SparkBuild extends Build { name := "spark-examples", libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm), - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") exclude("com.ning","compress-lzf") exclude("io.netty", "netty") exclude("jline","jline") - exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSnappy) - excludeAll(excludeCglib) + excludeAll(excludeSnappy, excludeCglib, excludeSLF4J) ) ) ++ assemblySettings ++ extraAssemblySettings @@ -416,7 +417,7 @@ object SparkBuild extends Build { exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty) + excludeAll(excludeNetty, excludeSLF4J) ) ) diff --git a/repl/pom.xml b/repl/pom.xml index 143b009f3c2a3..73597f635b9e0 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -72,10 +72,6 @@ org.slf4j jul-to-slf4j - - org.slf4j - slf4j-log4j12 - org.scalatest scalatest_${scala.binary.version} diff --git a/streaming/pom.xml b/streaming/pom.xml index d31ee60e4ea15..acc1d23c7dc20 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -79,11 +79,6 @@ junit-interface test - - org.slf4j - slf4j-log4j12 - test - commons-io commons-io From cd32d5e4dee1291e4509e5965322b7ffe620b1f3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Feb 2014 23:45:48 -0800 Subject: [PATCH 23/32] SPARK-1124: Fix infinite retries of reduce stage when a map stage failed In the previous code, if you had a failing map stage and then tried to run reduce stages on it repeatedly, the first reduce stage would fail correctly, but the later ones would mistakenly believe that all map outputs are available and start failing infinitely with fetch failures from "null". --- .../apache/spark/scheduler/DAGScheduler.scala | 31 ++++++++++--------- .../scala/org/apache/spark/FailureSuite.scala | 13 ++++++++ 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 729f518b89c06..789d5e6699907 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -272,8 +272,10 @@ class DAGScheduler( if (mapOutputTracker.has(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) - for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i)) - stage.numAvailableOutputs = locs.size + for (i <- 0 until locs.size) { + stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing + } + stage.numAvailableOutputs = locs.count(_ != null) } else { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -373,25 +375,26 @@ class DAGScheduler( } else { def removeStage(stageId: Int) { // data structures based on Stage - stageIdToStage.get(stageId).foreach { s => - if (running.contains(s)) { + for (stage <- stageIdToStage.get(stageId)) { + if (running.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) - running -= s + running -= stage + } + stageToInfos -= stage + for (shuffleDep <- stage.shuffleDep) { + shuffleToMapStage.remove(shuffleDep.shuffleId) } - stageToInfos -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleId => - shuffleToMapStage.remove(shuffleId)) - if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + if (pendingTasks.contains(stage) && !pendingTasks(stage).isEmpty) { logDebug("Removing pending status for stage %d".format(stageId)) } - pendingTasks -= s - if (waiting.contains(s)) { + pendingTasks -= stage + if (waiting.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= s + waiting -= stage } - if (failed.contains(s)) { + if (failed.contains(stage)) { logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= s + failed -= stage } } // data structures based on StageId diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index ac3c86778d526..f3fb64d87a2fd 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -81,6 +81,19 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } + // Run a map-reduce job in which the map stage always fails. + test("failure in a map stage") { + sc = new SparkContext("local", "test") + val data = sc.makeRDD(1 to 3).map(x => { throw new Exception; (x, x) }).groupByKey(3) + intercept[SparkException] { + data.collect() + } + // Make sure that running new jobs with the same map stage also fails + intercept[SparkException] { + data.collect() + } + } + test("failure because task results are not serializable") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) From 0187cef0f284e6cb22cb3986c327c43304daf57d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 24 Feb 2014 13:14:56 -0800 Subject: [PATCH 24/32] Fix removal from shuffleToMapStage to search for a key-value pair with our stage instead of using our shuffleID. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 789d5e6699907..dc5b25d845dc2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -381,8 +381,8 @@ class DAGScheduler( running -= stage } stageToInfos -= stage - for (shuffleDep <- stage.shuffleDep) { - shuffleToMapStage.remove(shuffleDep.shuffleId) + for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { + shuffleToMapStage.remove(k) } if (pendingTasks.contains(stage) && !pendingTasks(stage).isEmpty) { logDebug("Removing pending status for stage %d".format(stageId)) From 4d880304867b55a4f2138617b30600b7fa013b14 Mon Sep 17 00:00:00 2001 From: Bryn Keller Date: Mon, 24 Feb 2014 17:35:22 -0800 Subject: [PATCH 25/32] For outputformats that are Configurable, call setConf before sending data to them. [SPARK-1108] This allows us to use, e.g. HBase's TableOutputFormat with PairRDDFunctions.saveAsNewAPIHadoopFile, which otherwise would throw NullPointerException because the output table name hasn't been configured. Note this bug also affects branch-0.9 Author: Bryn Keller Closes #638 from xoltar/SPARK-1108 and squashes the following commits: 7e94e7d [Bryn Keller] Import, comment, and format cleanup per code review 7cbcaa1 [Bryn Keller] For outputformats that are Configurable, call setConf before sending data to them. This allows us to use, e.g. HBase TableOutputFormat, which otherwise would throw NullPointerException because the output table name hasn't been configured --- .../apache/spark/rdd/PairRDDFunctions.scala | 6 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 75 +++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 39c3a4996c327..d29a1a9881cd4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -29,7 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog -import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.Path import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec @@ -618,6 +618,10 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance + format match { + case c: Configurable => c.setConf(wrappedConf.value) + case _ => () + } val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index fa5c9b10fe059..e3e23775f011d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -23,6 +23,8 @@ import scala.util.Random import org.scalatest.FunSuite import com.google.common.io.Files +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.conf.{Configuration, Configurable} import org.apache.spark.SparkContext._ import org.apache.spark.{Partitioner, SharedSparkContext} @@ -330,4 +332,77 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { (1, ArrayBuffer(1)), (2, ArrayBuffer(1)))) } + + test("saveNewAPIHadoopFile should call setConf if format is configurable") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + + // No error, non-configurable formats still work + pairs.saveAsNewAPIHadoopFile[FakeFormat]("ignored") + + /* + Check that configurable formats get configured: + ConfigTestFormat throws an exception if we try to write + to it when setConf hasn't been called first. + Assertion is in ConfigTestFormat.getRecordWriter. + */ + pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") + } } + +/* + These classes are fakes for testing + "saveNewAPIHadoopFile should call setConf if format is configurable". + Unfortunately, they have to be top level classes, and not defined in + the test method, because otherwise Scala won't generate no-args constructors + and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile + tries to instantiate them with Class.newInstance. + */ +class FakeWriter extends RecordWriter[Integer, Integer] { + + def close(p1: TaskAttemptContext) = () + + def write(p1: Integer, p2: Integer) = () + +} + +class FakeCommitter extends OutputCommitter { + def setupJob(p1: JobContext) = () + + def needsTaskCommit(p1: TaskAttemptContext): Boolean = false + + def setupTask(p1: TaskAttemptContext) = () + + def commitTask(p1: TaskAttemptContext) = () + + def abortTask(p1: TaskAttemptContext) = () +} + +class FakeFormat() extends OutputFormat[Integer, Integer]() { + + def checkOutputSpecs(p1: JobContext) = () + + def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { + new FakeWriter() + } + + def getOutputCommitter(p1: TaskAttemptContext): OutputCommitter = { + new FakeCommitter() + } +} + +class ConfigTestFormat() extends FakeFormat() with Configurable { + + var setConfCalled = false + def setConf(p1: Configuration) = { + setConfCalled = true + () + } + + def getConf: Configuration = null + + override def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { + assert(setConfCalled, "setConf was never called") + super.getRecordWriter(p1) + } +} + From a4f4fbc8fa5886a8c6ee58ee614de0cc6e67dcd7 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 24 Feb 2014 21:13:38 -0800 Subject: [PATCH 26/32] Include reference to twitter/chill in tuning docs Author: Andrew Ash Closes #647 from ash211/doc-tuning and squashes the following commits: b87de0a [Andrew Ash] Include reference to twitter/chill in tuning docs --- docs/tuning.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 6b010aed618a3..704778681cb8f 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -44,7 +44,10 @@ This setting configures the serializer used for not only shuffling data between nodes but also when serializing RDDs to disk. The only reason Kryo is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. -Finally, to register your classes with Kryo, create a public class that extends +Spark automatically includes Kryo serializers for the many commonly-used core Scala classes covered +in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. + +To register your own custom classes with Kryo, create a public class that extends [`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` config property to point to it, as follows: @@ -72,8 +75,8 @@ If your objects are large, you may also need to increase the `spark.kryoserializ config property. The default is 2, but this value needs to be large enough to hold the *largest* object you will serialize. -Finally, if you don't register your classes, Kryo will still work, but it will have to store the -full class name with each object, which is wasteful. +Finally, if you don't register your custom classes, Kryo will still work, but it will have to store +the full class name with each object, which is wasteful. # Memory Tuning From 1f4c7f7ecc9d2393663fc4d059e71fe4c70bad84 Mon Sep 17 00:00:00 2001 From: Semih Salihoglu Date: Mon, 24 Feb 2014 22:42:30 -0800 Subject: [PATCH 27/32] Graph primitives2 Hi guys, I'm following Joey and Ankur's suggestions to add collectEdges and pickRandomVertex. I'm also adding the tests for collectEdges and refactoring one method getCycleGraph in GraphOpsSuite.scala. Thank you, semih Author: Semih Salihoglu Closes #580 from semihsalihoglu/GraphPrimitives2 and squashes the following commits: 937d3ec [Semih Salihoglu] - Fixed the scalastyle errors. a69a152 [Semih Salihoglu] - Adding collectEdges and pickRandomVertices. - Adding tests for collectEdges. - Refactoring a getCycle utility function for GraphOpsSuite.scala. 41265a6 [Semih Salihoglu] - Adding collectEdges and pickRandomVertex. - Adding tests for collectEdges. - Recycling a getCycle utility test file. --- .../org/apache/spark/graphx/GraphOps.scala | 59 +++++++- .../apache/spark/graphx/GraphOpsSuite.scala | 134 ++++++++++++++++-- 2 files changed, 183 insertions(+), 10 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 0fc1e4df6813c..377d9d6bd5e72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -18,11 +18,11 @@ package org.apache.spark.graphx import scala.reflect.ClassTag - import org.apache.spark.SparkContext._ import org.apache.spark.SparkException import org.apache.spark.graphx.lib._ import org.apache.spark.rdd.RDD +import scala.util.Random /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -137,6 +137,42 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali } } // end of collectNeighbor + /** + * Returns an RDD that contains for each vertex v its local edges, + * i.e., the edges that are incident on v, in the user-specified direction. + * Warning: note that singleton vertices, those with no edges in the given + * direction will not be part of the return value. + * + * @note This function could be highly inefficient on power-law + * graphs where high degree vertices may force a large amount of + * information to be collected to a single location. + * + * @param edgeDirection the direction along which to collect + * the local edges of vertices + * + * @return the local edges for each vertex + */ + def collectEdges(edgeDirection: EdgeDirection): VertexRDD[Array[Edge[ED]]] = { + edgeDirection match { + case EdgeDirection.Either => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.srcId, Array(new Edge(edge.srcId, edge.dstId, edge.attr))), + (edge.dstId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.In => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.dstId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.Out => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.srcId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.Both => + throw new SparkException("collectEdges does not support EdgeDirection.Both. Use" + + "EdgeDirection.Either instead.") + } + } + /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table @@ -209,6 +245,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali graph.mask(preprocess(graph).subgraph(epred, vpred)) } + /** + * Picks a random vertex from the graph and returns its ID. + */ + def pickRandomVertex(): VertexId = { + val probability = 50 / graph.numVertices + var found = false + var retVal: VertexId = null.asInstanceOf[VertexId] + while (!found) { + val selectedVertices = graph.vertices.flatMap { vidVvals => + if (Random.nextDouble() < probability) { Some(vidVvals._1) } + else { None } + } + if (selectedVertices.count > 1) { + found = true + val collectedVertices = selectedVertices.collect() + retVal = collectedVertices(Random.nextInt(collectedVertices.size)) + } + } + retVal + } + /** * Execute a Pregel-like iterative vertex-parallel abstraction. The * user-defined vertex-program `vprog` is executed in parallel on diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index bc2ad5677f806..6386306c048fc 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -42,21 +42,20 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { test("collectNeighborIds") { withSpark { sc => - val chain = (0 until 100).map(x => (x, (x+1)%100) ) - val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } - val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val graph = getCycleGraph(sc, 100) val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache() - assert(nbrs.count === chain.size) + assert(nbrs.count === 100) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } - nbrs.collect.foreach { case (vid, nbrs) => - val s = nbrs.toSet - assert(s.contains((vid + 1) % 100)) - assert(s.contains(if (vid > 0) vid - 1 else 99 )) + nbrs.collect.foreach { + case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) vid - 1 else 99)) } } } - + test ("filter") { withSpark { sc => val n = 5 @@ -80,4 +79,121 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } + test("collectEdgesCycleDirectionOut") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains((vid + 1) % 100)) + } + } + } + + test("collectEdgesCycleDirectionIn") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.In).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeSrcIds = s.map(e => e.srcId) + assert(edgeSrcIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesCycleDirectionEither") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 2) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + assert(edgeIds.contains((vid + 1) % 100)) + assert(edgeIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesChainDirectionOut") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count == 49) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains(vid + 1)) + } + } + } + + test("collectEdgesChainDirectionIn") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.In).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count == 49) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.srcId) + assert(edgeDstIds.contains((vid - 1) % 100)) + } + } + } + + test("collectEdgesChainDirectionEither") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count === 50) + edges.collect.foreach { + case (vid, edges) => if (vid > 0 && vid < 49) assert(edges.size == 2) + else assert(edges.size == 1) + } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + if (vid == 0) { assert(edgeIds.contains(1)) } + else if (vid == 49) { assert(edgeIds.contains(48)) } + else { + assert(edgeIds.contains(vid + 1)) + assert(edgeIds.contains(vid - 1)) + } + } + } + } + + private def getCycleGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val cycle = (0 until numVertices).map(x => (x, (x + 1) % numVertices)) + getGraphFromSeq(sc, cycle) + } + + private def getChainGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val chain = (0 until numVertices - 1).map(x => (x, (x + 1))) + getGraphFromSeq(sc, chain) + } + + private def getGraphFromSeq(sc: SparkContext, seq: IndexedSeq[(Int, Int)]): Graph[Double, Int] = { + val rawEdges = sc.parallelize(seq, 3).map { case (s, d) => (s.toLong, d.toLong) } + Graph.fromEdgeTuples(rawEdges, 1.0).cache() + } } From c852201ce95c7c982ff3794c114427eb33e92922 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 24 Feb 2014 23:20:38 -0800 Subject: [PATCH 28/32] For SPARK-1082, Use Curator for ZK interaction in standalone cluster Author: Raymond Liu Closes #611 from colorant/curator and squashes the following commits: 7556aa1 [Raymond Liu] Address review comments af92e1f [Raymond Liu] Fix coding style 964f3c2 [Raymond Liu] Ignore NodeExists exception 6df2966 [Raymond Liu] Rewrite zookeeper client code with curator --- core/pom.xml | 4 +- .../deploy/master/LeaderElectionAgent.scala | 1 + .../spark/deploy/master/MasterMessages.scala | 4 - .../deploy/master/SparkCuratorUtil.scala | 53 +++++ .../deploy/master/SparkZooKeeperSession.scala | 205 ------------------ .../master/ZooKeeperLeaderElectionAgent.scala | 94 +++----- .../master/ZooKeeperPersistenceEngine.scala | 30 +-- pom.xml | 6 +- project/SparkBuild.scala | 2 +- 9 files changed, 99 insertions(+), 300 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala delete mode 100644 core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala diff --git a/core/pom.xml b/core/pom.xml index 5576b0c3b4795..f209704f319de 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,8 +55,8 @@ avro-ipc - org.apache.zookeeper - zookeeper + org.apache.curator + curator-recipes org.eclipse.jetty diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index f25a1ad3bf92a..a730fe1f599af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -30,6 +30,7 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] */ private[spark] trait LeaderElectionAgent extends Actor { + //TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. val masterActor: ActorRef } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index 74a9f8cd824fb..db72d8ae9bdaf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -28,10 +28,6 @@ private[master] object MasterMessages { case object RevokedLeadership - // Actor System to LeaderElectionAgent - - case object CheckLeader - // Actor System to Master case object CheckForWorkerTimeOut diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala new file mode 100644 index 0000000000000..2d35397035a03 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master + +import org.apache.spark.{SparkConf, Logging} +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.zookeeper.KeeperException + + +object SparkCuratorUtil extends Logging { + + val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 + val ZK_SESSION_TIMEOUT_MILLIS = 60000 + val RETRY_WAIT_MILLIS = 5000 + val MAX_RECONNECT_ATTEMPTS = 3 + + def newClient(conf: SparkConf): CuratorFramework = { + val ZK_URL = conf.get("spark.deploy.zookeeper.url") + val zk = CuratorFrameworkFactory.newClient(ZK_URL, + ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, + new ExponentialBackoffRetry(RETRY_WAIT_MILLIS, MAX_RECONNECT_ATTEMPTS)) + zk.start() + zk + } + + def mkdir(zk: CuratorFramework, path: String) { + if (zk.checkExists().forPath(path) == null) { + try { + zk.create().creatingParentsIfNeeded().forPath(path) + } catch { + case nodeExist: KeeperException.NodeExistsException => + // do nothing, ignore node existing exception. + case e: Exception => throw e + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala deleted file mode 100644 index 57758055b19c0..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.master - -import scala.collection.JavaConversions._ - -import org.apache.zookeeper._ -import org.apache.zookeeper.Watcher.Event.KeeperState -import org.apache.zookeeper.data.Stat - -import org.apache.spark.{Logging, SparkConf} - -/** - * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry - * logic. If the ZooKeeper session expires or otherwise dies, a new ZooKeeper session will be - * created. If ZooKeeper remains down after several retries, the given - * [[org.apache.spark.deploy.master.SparkZooKeeperWatcher SparkZooKeeperWatcher]] will be - * informed via zkDown(). - * - * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many - * times or a semantic exception is thrown (e.g., "node already exists"). - */ -private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, - conf: SparkConf) extends Logging { - val ZK_URL = conf.get("spark.deploy.zookeeper.url", "") - - val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE - val ZK_TIMEOUT_MILLIS = 30000 - val RETRY_WAIT_MILLIS = 5000 - val ZK_CHECK_PERIOD_MILLIS = 10000 - val MAX_RECONNECT_ATTEMPTS = 3 - - private var zk: ZooKeeper = _ - - private val watcher = new ZooKeeperWatcher() - private var reconnectAttempts = 0 - private var closed = false - - /** Connect to ZooKeeper to start the session. Must be called before anything else. */ - def connect() { - connectToZooKeeper() - - new Thread() { - override def run() = sessionMonitorThread() - }.start() - } - - def sessionMonitorThread(): Unit = { - while (!closed) { - Thread.sleep(ZK_CHECK_PERIOD_MILLIS) - if (zk.getState != ZooKeeper.States.CONNECTED) { - reconnectAttempts += 1 - val attemptsLeft = MAX_RECONNECT_ATTEMPTS - reconnectAttempts - if (attemptsLeft <= 0) { - logError("Could not connect to ZooKeeper: system failure") - zkWatcher.zkDown() - close() - } else { - logWarning("ZooKeeper connection failed, retrying " + attemptsLeft + " more times...") - connectToZooKeeper() - } - } - } - } - - def close() { - if (!closed && zk != null) { zk.close() } - closed = true - } - - private def connectToZooKeeper() { - if (zk != null) zk.close() - zk = new ZooKeeper(ZK_URL, ZK_TIMEOUT_MILLIS, watcher) - } - - /** - * Attempts to maintain a live ZooKeeper exception despite (very) transient failures. - * Mainly useful for handling the natural ZooKeeper session expiration. - */ - private class ZooKeeperWatcher extends Watcher { - def process(event: WatchedEvent) { - if (closed) { return } - - event.getState match { - case KeeperState.SyncConnected => - reconnectAttempts = 0 - zkWatcher.zkSessionCreated() - case KeeperState.Expired => - connectToZooKeeper() - case KeeperState.Disconnected => - logWarning("ZooKeeper disconnected, will retry...") - case s => // Do nothing - } - } - } - - def create(path: String, bytes: Array[Byte], createMode: CreateMode): String = { - retry { - zk.create(path, bytes, ZK_ACL, createMode) - } - } - - def exists(path: String, watcher: Watcher = null): Stat = { - retry { - zk.exists(path, watcher) - } - } - - def getChildren(path: String, watcher: Watcher = null): List[String] = { - retry { - zk.getChildren(path, watcher).toList - } - } - - def getData(path: String): Array[Byte] = { - retry { - zk.getData(path, false, null) - } - } - - def delete(path: String, version: Int = -1): Unit = { - retry { - zk.delete(path, version) - } - } - - /** - * Creates the given directory (non-recursively) if it doesn't exist. - * All znodes are created in PERSISTENT mode with no data. - */ - def mkdir(path: String) { - if (exists(path) == null) { - try { - create(path, "".getBytes, CreateMode.PERSISTENT) - } catch { - case e: Exception => - // If the exception caused the directory not to be created, bubble it up, - // otherwise ignore it. - if (exists(path) == null) { throw e } - } - } - } - - /** - * Recursively creates all directories up to the given one. - * All znodes are created in PERSISTENT mode with no data. - */ - def mkdirRecursive(path: String) { - var fullDir = "" - for (dentry <- path.split("/").tail) { - fullDir += "/" + dentry - mkdir(fullDir) - } - } - - /** - * Retries the given function up to 3 times. The assumption is that failure is transient, - * UNLESS it is a semantic exception (i.e., trying to get data from a node that doesn't exist), - * in which case the exception will be thrown without retries. - * - * @param fn Block to execute, possibly multiple times. - */ - def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = { - try { - fn - } catch { - case e: KeeperException.NoNodeException => throw e - case e: KeeperException.NodeExistsException => throw e - case e: Exception if n > 0 => - logError("ZooKeeper exception, " + n + " more retries...", e) - Thread.sleep(RETRY_WAIT_MILLIS) - retry(fn, n-1) - } - } -} - -trait SparkZooKeeperWatcher { - /** - * Called whenever a ZK session is created -- - * this will occur when we create our first session as well as each time - * the session expires or errors out. - */ - def zkSessionCreated() - - /** - * Called if ZK appears to be completely down (i.e., not just a transient error). - * We will no longer attempt to reconnect to ZK, and the SparkZooKeeperSession is considered dead. - */ - def zkDown() -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 47b8f67f8a45b..285f9b014e291 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -18,105 +18,67 @@ package org.apache.spark.deploy.master import akka.actor.ActorRef -import org.apache.zookeeper._ -import org.apache.zookeeper.Watcher.Event.EventType import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.master.MasterMessages._ +import org.apache.curator.framework.CuratorFramework +import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) - extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { + extends LeaderElectionAgent with LeaderLatchListener with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" - private val watcher = new ZooKeeperWatcher() - private val zk = new SparkZooKeeperSession(this, conf) + private var zk: CuratorFramework = _ + private var leaderLatch: LeaderLatch = _ private var status = LeadershipStatus.NOT_LEADER - private var myLeaderFile: String = _ - private var leaderUrl: String = _ override def preStart() { + logInfo("Starting ZooKeeper LeaderElection agent") - zk.connect() - } + zk = SparkCuratorUtil.newClient(conf) + leaderLatch = new LeaderLatch(zk, WORKING_DIR) + leaderLatch.addListener(this) - override def zkSessionCreated() { - synchronized { - zk.mkdirRecursive(WORKING_DIR) - myLeaderFile = - zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL) - self ! CheckLeader - } + leaderLatch.start() } override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) { - logError("LeaderElectionAgent failed, waiting " + zk.ZK_TIMEOUT_MILLIS + "...", reason) - Thread.sleep(zk.ZK_TIMEOUT_MILLIS) + logError("LeaderElectionAgent failed...", reason) super.preRestart(reason, message) } - override def zkDown() { - logError("ZooKeeper down! LeaderElectionAgent shutting down Master.") - System.exit(1) - } - override def postStop() { + leaderLatch.close() zk.close() } override def receive = { - case CheckLeader => checkLeader() + case _ => } - private class ZooKeeperWatcher extends Watcher { - def process(event: WatchedEvent) { - if (event.getType == EventType.NodeDeleted) { - logInfo("Leader file disappeared, a master is down!") - self ! CheckLeader + override def isLeader() { + synchronized { + // could have lost leadership by now. + if (!leaderLatch.hasLeadership) { + return } - } - } - /** Uses ZK leader election. Navigates several ZK potholes along the way. */ - def checkLeader() { - val masters = zk.getChildren(WORKING_DIR).toList - val leader = masters.sorted.head - val leaderFile = WORKING_DIR + "/" + leader - - // Setup a watch for the current leader. - zk.exists(leaderFile, watcher) - - try { - leaderUrl = new String(zk.getData(leaderFile)) - } catch { - // A NoNodeException may be thrown if old leader died since the start of this method call. - // This is fine -- just check again, since we're guaranteed to see the new values. - case e: KeeperException.NoNodeException => - logInfo("Leader disappeared while reading it -- finding next leader") - checkLeader() - return + logInfo("We have gained leadership") + updateLeadershipStatus(true) } + } - // Synchronization used to ensure no interleaving between the creation of a new session and the - // checking of a leader, which could cause us to delete our real leader file erroneously. + override def notLeader() { synchronized { - val isLeader = myLeaderFile == leaderFile - if (!isLeader && leaderUrl == masterUrl) { - // We found a different master file pointing to this process. - // This can happen in the following two cases: - // (1) The master process was restarted on the same node. - // (2) The ZK server died between creating the file and returning the name of the file. - // For this case, we will end up creating a second file, and MUST explicitly delete the - // first one, since our ZK session is still open. - // Note that this deletion will cause a NodeDeleted event to be fired so we check again for - // leader changes. - assert(leaderFile < myLeaderFile) - logWarning("Cleaning up old ZK master election file that points to this master.") - zk.delete(leaderFile) - } else { - updateLeadershipStatus(isLeader) + // could have gained leadership by now. + if (leaderLatch.hasLeadership) { + return } + + logInfo("We have lost leadership") + updateLeadershipStatus(false) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 48b2fc06a9d70..939006239d2b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,36 +17,28 @@ package org.apache.spark.deploy.master +import scala.collection.JavaConversions._ + import akka.serialization.Serialization -import org.apache.zookeeper._ +import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine - with SparkZooKeeperWatcher with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val zk = SparkCuratorUtil.newClient(conf) - val zk = new SparkZooKeeperSession(this, conf) - - zk.connect() - - override def zkSessionCreated() { - zk.mkdirRecursive(WORKING_DIR) - } - - override def zkDown() { - logError("PersistenceEngine disconnected from ZooKeeper -- ZK looks down.") - } + SparkCuratorUtil.mkdir(zk, WORKING_DIR) override def addApplication(app: ApplicationInfo) { serializeIntoFile(WORKING_DIR + "/app_" + app.id, app) } override def removeApplication(app: ApplicationInfo) { - zk.delete(WORKING_DIR + "/app_" + app.id) + zk.delete().forPath(WORKING_DIR + "/app_" + app.id) } override def addDriver(driver: DriverInfo) { @@ -54,7 +46,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def removeDriver(driver: DriverInfo) { - zk.delete(WORKING_DIR + "/driver_" + driver.id) + zk.delete().forPath(WORKING_DIR + "/driver_" + driver.id) } override def addWorker(worker: WorkerInfo) { @@ -62,7 +54,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def removeWorker(worker: WorkerInfo) { - zk.delete(WORKING_DIR + "/worker_" + worker.id) + zk.delete().forPath(WORKING_DIR + "/worker_" + worker.id) } override def close() { @@ -70,7 +62,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { - val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted + val sortedFiles = zk.getChildren().forPath(WORKING_DIR).toList.sorted val appFiles = sortedFiles.filter(_.startsWith("app_")) val apps = appFiles.map(deserializeFromFile[ApplicationInfo]) val driverFiles = sortedFiles.filter(_.startsWith("driver_")) @@ -83,11 +75,11 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) private def serializeIntoFile(path: String, value: AnyRef) { val serializer = serialization.findSerializerFor(value) val serialized = serializer.toBinary(value) - zk.create(path, serialized, CreateMode.PERSISTENT) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = { - val fileData = zk.getData(WORKING_DIR + "/" + filename) + val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) serializer.fromBinary(fileData).asInstanceOf[T] diff --git a/pom.xml b/pom.xml index 3a530685b8e5a..4f1e8398d9b8d 100644 --- a/pom.xml +++ b/pom.xml @@ -393,9 +393,9 @@ test - org.apache.zookeeper - zookeeper - 3.4.5 + org.apache.curator + curator-recipes + 2.4.0 org.jboss.netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f0d2e741484f9..220894affb4da 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -277,7 +277,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), "org.apache.avro" % "avro" % "1.7.4", "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), - "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", From b8a1871953058c67b49b7f8455cbb417d5b50ab6 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 26 Feb 2014 10:00:02 -0600 Subject: [PATCH 29/32] SPARK-1053. Don't require SPARK_YARN_APP_JAR It looks this just requires taking out the checks. I verified that, with the patch, I was able to run spark-shell through yarn without setting the environment variable. Author: Sandy Ryza Closes #553 from sryza/sandy-spark-1053 and squashes the following commits: b037676 [Sandy Ryza] SPARK-1053. Don't require SPARK_YARN_APP_JAR --- docs/running-on-yarn.md | 6 ++---- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 4 ++-- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 6 +----- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index cd4509ede735a..ee1d892a3b630 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -99,13 +99,12 @@ With this mode, your application is actually run on the remote machine where the ## Launch spark application with yarn-client mode. -With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR +With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR. Configuration in yarn-client mode: In order to tune worker core/number/memory etc. You need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. -* `SPARK_YARN_APP_JAR`, Path to your application's JAR file (required) * `SPARK_WORKER_INSTANCES`, Number of workers to start (Default: 2) * `SPARK_WORKER_CORES`, Number of cores for the workers (Default: 1). * `SPARK_WORKER_MEMORY`, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) @@ -118,12 +117,11 @@ In order to tune worker core/number/memory etc. You need to export environment v For example: SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ ./bin/run-example org.apache.spark.examples.SparkPi yarn-client +or SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./bin/spark-shell diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 1419f215c78e5..fe37168e5a7ba 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -108,7 +108,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { args = tail case Nil => - if (userJar == null || userClass == null) { + if (userClass == null) { printUsageAndExit(1) } @@ -129,7 +129,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { System.err.println( "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --jar JAR_PATH Path to your application's JAR file (required in yarn-standalone mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 2db5744be1a70..24520bd21ba98 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -68,7 +68,8 @@ trait ClientBase extends Logging { def validateArgs() = { Map( (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", - (args.userJar == null) -> "Error: You must specify a user jar!", + ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> + "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 22e55e0c60647..e7130d24072ca 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -44,10 +44,6 @@ private[spark] class YarnClientSchedulerBackend( override def start() { super.start() - val userJar = System.getenv("SPARK_YARN_APP_JAR") - if (userJar == null) - throw new SparkException("env SPARK_YARN_APP_JAR is not set") - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort @@ -55,7 +51,7 @@ private[spark] class YarnClientSchedulerBackend( val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( "--class", "notused", - "--jar", userJar, + "--jar", null, "--args", hostport, "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" ) From fbedc8eff2573b31320ad1b11a1826a2e530c16c Mon Sep 17 00:00:00 2001 From: William Benton Date: Wed, 26 Feb 2014 10:09:50 -0800 Subject: [PATCH 30/32] SPARK-1078: Replace lift-json with json4s-jackson. The aim of the Json4s project is to provide a common API for Scala JSON libraries. It is Apache-licensed, easier for downstream distributions to package, and mostly API-compatible with lift-json. Furthermore, the Jackson-backed implementation parses faster than lift-json on all but the smallest inputs. Author: William Benton Closes #582 from willb/json4s and squashes the following commits: 7ca62c4 [William Benton] Replace lift-json with json4s-jackson. --- core/pom.xml | 5 ++-- .../spark/deploy/FaultToleranceTest.scala | 9 ++++--- .../apache/spark/deploy/JsonProtocol.scala | 2 +- .../deploy/master/ui/ApplicationPage.scala | 3 ++- .../spark/deploy/master/ui/IndexPage.scala | 3 ++- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../org/apache/spark/ui/JettyUtils.scala | 3 ++- .../spark/deploy/JsonProtocolSuite.scala | 27 ++++++++++--------- project/SparkBuild.scala | 2 +- 9 files changed, 32 insertions(+), 24 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index f209704f319de..ebc178a10541a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -130,8 +130,9 @@ scala-library - net.liftweb - lift-json_${scala.binary.version} + org.json4s + json4s-jackson_${scala.binary.version} + 3.2.6 it.unimi.dsi diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 190b331cfe7d8..d48c1892aea9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -27,7 +27,8 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.sys.process._ -import net.liftweb.json.JsonParser +import org.json4s._ +import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.master.RecoveryState @@ -311,7 +312,7 @@ private[spark] object FaultToleranceTest extends App with Logging { private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { - implicit val formats = net.liftweb.json.DefaultFormats + implicit val formats = org.json4s.DefaultFormats var state: RecoveryState.Value = _ var liveWorkerIPs: List[String] = _ var numLiveApps = 0 @@ -321,7 +322,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val def readState() { try { val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream) - val json = JsonParser.parse(masterStream, closeAutomatically = true) + val json = JsonMethods.parse(masterStream) val workers = json \ "workers" val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE") @@ -349,7 +350,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { - implicit val formats = net.liftweb.json.DefaultFormats + implicit val formats = org.json4s.DefaultFormats logDebug("Created worker: " + this) diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 318beb5db5214..cefb1ff97e83c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import net.liftweb.json.JsonDSL._ +import org.json4s.JsonDSL._ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 5cc4adbe448b7..90cad3c37fda6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,7 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import net.liftweb.json.JsonAST.JValue +import javax.servlet.http.HttpServletRequest +import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 01c8f9065e50a..bac922bcd3d98 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -23,7 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import net.liftweb.json.JsonAST.JValue +import javax.servlet.http.HttpServletRequest +import org.json4s.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 3089acffb8d98..85200ab0e102d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -22,7 +22,7 @@ import scala.xml.Node import akka.pattern.ask import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue +import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 1f048a84cdfb6..1b78c52ff6077 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -24,7 +24,8 @@ import scala.annotation.tailrec import scala.util.{Failure, Success, Try} import scala.xml.Node -import net.liftweb.json.{JValue, pretty, render} +import org.json4s.JValue +import org.json4s.jackson.JsonMethods.{pretty, render} import org.eclipse.jetty.server.{Handler, Request, Server} import org.eclipse.jetty.server.handler.{AbstractHandler, ContextHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index de866ed7ffed8..bae3b37e267d5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.deploy import java.io.File import java.util.Date -import net.liftweb.json.Diff -import net.liftweb.json.{JsonAST, JsonParser} -import net.liftweb.json.JsonAST.{JNothing, JValue} +import org.json4s._ + +import org.json4s.JValue +import org.json4s.jackson.JsonMethods +import com.fasterxml.jackson.core.JsonParseException + import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -34,31 +37,31 @@ class JsonProtocolSuite extends FunSuite { test("writeApplicationInfo") { val output = JsonProtocol.writeApplicationInfo(createAppInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.appInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.appInfoJsonStr)) } test("writeWorkerInfo") { val output = JsonProtocol.writeWorkerInfo(createWorkerInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.workerInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.workerInfoJsonStr)) } test("writeApplicationDescription") { val output = JsonProtocol.writeApplicationDescription(createAppDesc()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.appDescJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.appDescJsonStr)) } test("writeExecutorRunner") { val output = JsonProtocol.writeExecutorRunner(createExecutorRunner()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.executorRunnerJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.executorRunnerJsonStr)) } test("writeDriverInfo") { val output = JsonProtocol.writeDriverInfo(createDriverInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.driverInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.driverInfoJsonStr)) } test("writeMasterState") { @@ -71,7 +74,7 @@ class JsonProtocolSuite extends FunSuite { activeDrivers, completedDrivers, RecoveryState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.masterStateJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.masterStateJsonStr)) } test("writeWorkerState") { @@ -83,7 +86,7 @@ class JsonProtocolSuite extends FunSuite { finishedExecutors, drivers, finishedDrivers, "masterUrl", 4, 1234, 4, 1234, "masterWebUiUrl") val output = JsonProtocol.writeWorkerState(stateResponse) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.workerStateJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.workerStateJsonStr)) } def createAppDesc(): ApplicationDescription = { @@ -125,9 +128,9 @@ class JsonProtocolSuite extends FunSuite { def assertValidJson(json: JValue) { try { - JsonParser.parse(JsonAST.compactRender(json)) + JsonMethods.parse(JsonMethods.compact(json)) } catch { - case e: JsonParser.ParseException => fail("Invalid Json detected", e) + case e: JsonParseException => fail("Invalid Json detected", e) } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 220894affb4da..7d157160d164b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -268,7 +268,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", - "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty), + "org.json4s" %% "json4s-jackson" % "3.2.6", "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", From c86eec584312072e73754a4f1cbe34d2e1968c77 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Feb 2014 11:20:16 -0800 Subject: [PATCH 31/32] SPARK-1135: fix broken anchors in docs A recent PR that added Java vs Scala tabs for streaming also inadvertently added some bad code to a document.ready handler, breaking our other handler that manages scrolling to anchors correctly with the floating top bar. As a result the section title ended up always being hidden below the top bar. This removes the unnecessary JavaScript code. Author: Matei Zaharia Closes #3 from mateiz/doc-links and squashes the following commits: e2a3488 [Matei Zaharia] SPARK-1135: fix broken anchors in docs --- docs/js/main.js | 29 +---------------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/docs/js/main.js b/docs/js/main.js index 102699789a71a..0bd2286cced19 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -1,26 +1,3 @@ - -// From docs.scala-lang.org -function styleCode() { - if (typeof disableStyleCode != "undefined") { - return; - } - $(".codetabs pre code").parent().each(function() { - if (!$(this).hasClass("prettyprint")) { - var lang = $(this).parent().data("lang"); - if (lang == "python") { - lang = "py" - } - if (lang == "bash") { - lang = "bsh" - } - $(this).addClass("prettyprint lang-"+lang+" linenums"); - } - }); - console.log("runningPrettyPrint()") - prettyPrint(); -} - - function codeTabs() { var counter = 0; var langImages = { @@ -97,11 +74,7 @@ function viewSolution() { } -$(document).ready(function() { +$(function() { codeTabs(); viewSolution(); - $('#chapter-toc').toc({exclude: '', context: '.container'}); - $('#chapter-toc').prepend('

In This Chapter

'); - makeCollapsable($('#global-toc'), "", "global-toc", "Show Table of Contents"); - //styleCode(); }); From 12738c1aec136acd7f2e3e2f8f2b541db0890630 Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Wed, 26 Feb 2014 14:50:37 -0800 Subject: [PATCH 32/32] SPARK-1115: Catch depickling errors This surroungs the complete worker code in a try/except block so we catch any error that arrives. An example would be the depickling failing for some reason @JoshRosen Author: Bouke van der Bijl Closes #644 from bouk/catch-depickling-errors and squashes the following commits: f0f67cc [Bouke van der Bijl] Lol indentation 0e4d504 [Bouke van der Bijl] Surround the complete python worker with the try block --- python/pyspark/worker.py | 48 ++++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 158646352039f..4c214ef359685 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -45,34 +45,34 @@ def report_times(outfile, boot, init, finish): def main(infile, outfile): - boot_time = time.time() - split_index = read_int(infile) - if split_index == -1: # for unit tests - return + try: + boot_time = time.time() + split_index = read_int(infile) + if split_index == -1: # for unit tests + return - # fetch name of workdir - spark_files_dir = utf8_deserializer.loads(infile) - SparkFiles._root_directory = spark_files_dir - SparkFiles._is_running_on_worker = True + # fetch name of workdir + spark_files_dir = utf8_deserializer.loads(infile) + SparkFiles._root_directory = spark_files_dir + SparkFiles._is_running_on_worker = True - # fetch names and values of broadcast variables - num_broadcast_variables = read_int(infile) - for _ in range(num_broadcast_variables): - bid = read_long(infile) - value = pickleSer._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) + # fetch names and values of broadcast variables + num_broadcast_variables = read_int(infile) + for _ in range(num_broadcast_variables): + bid = read_long(infile) + value = pickleSer._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) - # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here - num_python_includes = read_int(infile) - for _ in range(num_python_includes): - filename = utf8_deserializer.loads(infile) - sys.path.append(os.path.join(spark_files_dir, filename)) + # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH + sys.path.append(spark_files_dir) # *.py files that were added will be copied here + num_python_includes = read_int(infile) + for _ in range(num_python_includes): + filename = utf8_deserializer.loads(infile) + sys.path.append(os.path.join(spark_files_dir, filename)) - command = pickleSer._read_with_length(infile) - (func, deserializer, serializer) = command - init_time = time.time() - try: + command = pickleSer._read_with_length(infile) + (func, deserializer, serializer) = command + init_time = time.time() iterator = deserializer.load_stream(infile) serializer.dump_stream(func(split_index, iterator), outfile) except Exception as e: