diff --git a/assembly/pom.xml b/assembly/pom.xml index dacae92d83a8e..82396040251d3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.9.0-incubating-SNAPSHOT + 1.0.0-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index cb8e79f22535b..6155ab5f20155 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.9.0-incubating-SNAPSHOT + 1.0.0-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 44e26bbb9e094..dd3eed8affe39 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -28,21 +28,23 @@ object Bagel extends Logging { /** * Runs a Bagel program. * @param sc [[org.apache.spark.SparkContext]] to use for the program. - * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be - * the vertex id. - * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an - * empty array, i.e. sc.parallelize(Array[K, Message]()). - * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one - * message before sending (which often involves network I/O). - * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep, - * and provides the result to each vertex in the next superstep. + * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the + * Key will be the vertex id. + * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often + * this will be an empty array, i.e. sc.parallelize(Array[K, Message]()). + * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a + * given vertex into one message before sending (which often involves network + * I/O). + * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices + * after each superstep and provides the result to each vertex in the next + * superstep. * @param partitioner [[org.apache.spark.Partitioner]] partitions values by key * @param numPartitions number of partitions across which to split the graph. * Default is the default parallelism of the SparkContext - * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep. - * Defaults to caching in memory. - * @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex, - * optional Aggregator and the current superstep, + * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of + * intermediate RDDs in each superstep. Defaults to caching in memory. + * @param compute function that takes a Vertex, optional set of (possibly combined) messages to + * the Vertex, optional Aggregator and the current superstep, * and returns a set of (Vertex, outgoing Messages) pairs * @tparam K key * @tparam V vertex type @@ -71,7 +73,7 @@ object Bagel extends Logging { var msgs = messages var noActivity = false do { - logInfo("Starting superstep "+superstep+".") + logInfo("Starting superstep " + superstep + ".") val startTime = System.currentTimeMillis val aggregated = agg(verts, aggregator) @@ -97,7 +99,8 @@ object Bagel extends Logging { verts } - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default storage level */ + /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default + * storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -106,8 +109,8 @@ object Bagel extends Logging { partitioner: Partitioner, numPartitions: Int )( - compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) + compute: (V, Option[C], Int) => (V, Array[M])): RDD[(K, V)] = run(sc, vertices, messages, + combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( @@ -127,8 +130,8 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]] - * and default storage level + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default + * [[org.apache.spark.HashPartitioner]] and default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, @@ -138,9 +141,13 @@ object Bagel extends Logging { numPartitions: Int )( compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) + ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, + DEFAULT_STORAGE_LEVEL)(compute) - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default [[org.apache.spark.HashPartitioner]]*/ + /** + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the + * default [[org.apache.spark.HashPartitioner]] + */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -158,7 +165,8 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]], + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], + * default [[org.apache.spark.HashPartitioner]], * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( @@ -171,7 +179,8 @@ object Bagel extends Logging { ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], the default [[org.apache.spark.HashPartitioner]] + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], + * the default [[org.apache.spark.HashPartitioner]] * and [[org.apache.spark.bagel.DefaultCombiner]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( @@ -227,8 +236,9 @@ object Bagel extends Logging { }) numMsgs += newMsgs.size - if (newVert.active) + if (newVert.active) { numActiveVerts += 1 + } Some((newVert, newMsgs)) }.persist(storageLevel) 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/conf/metrics.properties.template b/conf/metrics.properties.template index 1c3d94e1b0831..30bcab0c93302 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -67,7 +67,7 @@ # period 10 Poll period # unit seconds Units of poll period # ttl 1 TTL of messages sent by Ganglia -# mode multicast Ganglia network mode ('unicast' or 'mulitcast') +# mode multicast Ganglia network mode ('unicast' or 'multicast') # org.apache.spark.metrics.sink.JmxSink 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/core/pom.xml b/core/pom.xml index 9e5a450d57a47..5576b0c3b4795 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.9.0-incubating-SNAPSHOT + 1.0.0-incubating-SNAPSHOT ../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/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/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..d5f3e3f6ec496 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 /** @@ -188,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/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 8e5dd8a85020d..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. @@ -31,8 +31,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private val loading = new HashSet[RDDBlockId]() /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ - def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel) - : Iterator[T] = { + def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, + storageLevel: StorageLevel): Iterator[T] = { val key = RDDBlockId(rdd.id, split.index) logDebug("Looking for partition " + key) blockManager.get(key) match { diff --git a/core/src/main/scala/org/apache/spark/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/FetchFailedException.scala index d242047502fd3..8eaa26bdb1b5b 100644 --- a/core/src/main/scala/org/apache/spark/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/FetchFailedException.scala @@ -25,7 +25,8 @@ private[spark] class FetchFailedException( cause: Throwable) extends Exception { - def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, cause: Throwable) = + def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, + cause: Throwable) = this(FetchFailed(bmAddress, shuffleId, mapId, reduceId), "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId), cause) 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 30d182b008930..5968973132942 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -22,14 +22,13 @@ 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 import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index cfba43dec3111..ad9988226470c 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.reflect.ClassTag import org.apache.spark.rdd.RDD +import org.apache.spark.util.CollectionsUtils import org.apache.spark.util.Utils /** @@ -118,12 +119,26 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( def numPartitions = partitions + private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] + def getPartition(key: Any): Int = { - // TODO: Use a binary search here if number of partitions is large val k = key.asInstanceOf[K] var partition = 0 - while (partition < rangeBounds.length && k > rangeBounds(partition)) { - partition += 1 + if (rangeBounds.length < 1000) { + // If we have less than 100 partitions naive search + while (partition < rangeBounds.length && k > rangeBounds(partition)) { + partition += 1 + } + } else { + // Determine which binary search method to use only once. + partition = binarySearch(rangeBounds, k) + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition-1 + } + if (partition > rangeBounds.length) { + partition = rangeBounds.length + } } if (ascending) { partition 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 566472e597958..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 @@ -63,9 +58,9 @@ import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, Metada */ class SparkContext( config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) - // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains - // a map from hostname to a list of input format splits on the host. + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) extends Logging { @@ -170,6 +165,11 @@ class SparkContext( .map(Utils.memoryStringToMb) .getOrElse(512) + if (!conf.contains("spark.executor.memory") && sys.env.contains("SPARK_MEM")) { + logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + + "deprecated, instead use spark.executor.memory") + } + // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner @@ -552,10 +552,11 @@ class SparkContext( /** * Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and - * BytesWritable values that contain a serialized partition. This is still an experimental storage - * format and may not be supported exactly as is in future Spark releases. It will also be pretty - * slow if you use the default serializer (Java serialization), though the nice thing about it is - * that there's very little effort required to save arbitrary objects. + * BytesWritable values that contain a serialized partition. This is still an experimental + * storage format and may not be supported exactly as is in future Spark releases. It will also + * be pretty slow if you use the default serializer (Java serialization), + * though the nice thing about it is that there's very little effort required to save arbitrary + * objects. */ def objectFile[T: ClassTag]( path: String, @@ -1043,7 +1044,7 @@ object SparkContext { implicit object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0l + def zero(initialValue: Long) = 0L } implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { @@ -1109,7 +1110,8 @@ object SparkContext { implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get) - implicit def booleanWritableConverter() = simpleWritableConverter[Boolean, BooleanWritable](_.get) + implicit def booleanWritableConverter() = + simpleWritableConverter[Boolean, BooleanWritable](_.get) implicit def bytesWritableConverter() = { simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) @@ -1258,7 +1260,8 @@ object SparkContext { case "yarn-client" => val scheduler = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] @@ -1269,7 +1272,8 @@ object SparkContext { } val backend = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ed788560e79f1..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), @@ -96,7 +95,7 @@ object SparkEnv extends Logging { @volatile private var lastSetSparkEnv : SparkEnv = _ def set(e: SparkEnv) { - lastSetSparkEnv = e + lastSetSparkEnv = e env.set(e) } @@ -112,7 +111,7 @@ object SparkEnv extends Logging { * Returns the ThreadLocal SparkEnv. */ def getThreadLocal: SparkEnv = { - env.get() + env.get() } private[spark] def create( @@ -168,7 +167,8 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf)), conf) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + serializer, conf) val connectionManager = blockManager.connectionManager 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/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 b0dedc6f4eb13..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,44 +143,44 @@ 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 count - * of the RDD's elements in one operation. + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * count of the RDD's elements in one operation. */ 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/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index f430a33db1e4a..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) @@ -88,7 +82,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numPartitions)) + def distinct(numPartitions: Int): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.distinct(numPartitions)) /** * Return a new RDD containing only the elements that satisfy a predicate. @@ -157,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, @@ -210,25 +205,25 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g ., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = - fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) + def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]) + : JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g ., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" + * which may be added to the result an arbitrary number of times, and must not change the result + * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue)(func)) @@ -276,6 +271,29 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def subtract(other: JavaPairRDD[K, V], p: Partitioner): JavaPairRDD[K, V] = fromRDD(rdd.subtract(other, p)) + /** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtractByKey[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, V] = { + 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 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 ctag: ClassTag[W] = fakeClassTag + fromRDD(rdd.subtractByKey(other, p)) + } + /** * Return a copy of the RDD partitioned using the specified partitioner. */ @@ -321,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))) } @@ -375,7 +393,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (V, Optional[W])] = { val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -397,7 +416,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (Optional[V], W)] = { val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -412,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)) } @@ -423,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)) } @@ -439,8 +459,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], + partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** @@ -462,8 +482,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (JList[V], JList[W])] = + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a @@ -655,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..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,12 +20,12 @@ 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]) 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 4db7339e6716b..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 @@ -76,7 +77,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + preservesPartitioning)) /** * Return a new RDD by applying a function to all elements of this RDD. @@ -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()) } /** @@ -134,7 +135,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { + def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], + preservesPartitioning: Boolean): JavaRDD[U] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) JavaRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning)(f.elementType()))(f.elementType()) } @@ -160,16 +162,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { + def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]], + preservesPartitioning: Boolean): JavaDoubleRDD = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning).map((x: java.lang.Double) => x.doubleValue())) + new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) + .map((x: java.lang.Double) => x.doubleValue())) } /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): - JavaPairRDD[K2, V2] = { + def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], + preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) JavaPairRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning))(f.keyType(), f.valueType()) } @@ -199,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))) } /** @@ -210,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))) } /** @@ -294,7 +296,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Reduces the elements of this RDD using the specified commutative and associative binary operator. + * Reduces the elements of this RDD using the specified commutative and associative binary + * operator. */ def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f) @@ -403,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 5a426b983519c..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. */ @@ -362,15 +360,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork doubleAccumulator(initialValue) /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) /** - * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks can - * "add" values with `add`. Only the master can access the accumuable's `value`. + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumuable's `value`. */ def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) @@ -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]] } 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 3e85052cd03e8..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,17 +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 5e9b8c48b89aa..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,17 +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 537439ef53888..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,22 +15,17 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; - -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; - -import java.io.Serializable; +package org.apache.spark.api.java.function +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 a2d1214fb4692..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 scala.reflect.ClassTag; -import scala.reflect.ClassTag$; - -import java.io.Serializable; +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 68% 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 fb1deceab5b56..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,22 +15,14 @@ * limitations under the License. */ -package org.apache.spark.api.java.function; +package org.apache.spark.api.java.function -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction2; - -import java.io.Serializable; +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 ca485b3cc2d93..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 scala.Tuple2; -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; - -import java.io.Serializable; +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 cbe2306026a61..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 scala.Tuple2; -import scala.reflect.ClassTag; -import scala.reflect.ClassTag$; - -import java.io.Serializable; +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/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 2be4e323bec98..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,13 +17,12 @@ package org.apache.spark.api.python -import java.util.Arrays - import org.apache.spark.Partitioner import org.apache.spark.util.Utils /** - * A [[org.apache.spark.Partitioner]] that performs handling of long-valued keys, for use by the Python API. + * A [[org.apache.spark.Partitioner]] that performs handling of long-valued keys, for use by the + * Python API. * * Stores the unique id() of the Python-side partitioning function so that it is incorporated into * equality comparisons. Correctness requires that the id is a unique identifier for the 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 da9ad783914de..ca0e702addddd 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 @@ -94,8 +94,9 @@ private[spark] class PythonRDD[T: ClassTag]( // Kill the Python worker process: worker.shutdownOutput() case e: IOException => - // This can happen for legitimate reasons if the Python code stops returning data before we are done - // passing elements through, e.g., for take(). Just log a message to say it happened. + // This can happen for legitimate reasons if the Python code stops returning data + // before we are done passing elements through, e.g., for take(). Just log a message + // to say it happened. logInfo("stdin writer to Python finished early") logDebug("stdin writer to Python finished early", e) } @@ -135,7 +136,8 @@ private[spark] class PythonRDD[T: ClassTag]( val init = initTime - bootTime val finish = finishTime - initTime val total = finishTime - startTime - logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, + init, finish)) read case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python @@ -187,7 +189,7 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (Utils.deserializeLongValue(a), b) - case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) + case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) } val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this) } @@ -417,7 +419,8 @@ private[spark] object PythonRDD extends Logging { } } -private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { +private +class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") } 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 d351dfc1f56a2..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 { @@ -187,8 +186,9 @@ extends Logging { val bais = new ByteArrayInputStream(byteArray) var blockNum = (byteArray.length / BLOCK_SIZE) - if (byteArray.length % BLOCK_SIZE != 0) + if (byteArray.length % BLOCK_SIZE != 0) { blockNum += 1 + } var retVal = new Array[TorrentBlock](blockNum) var blockID = 0 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/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index db67c6d1bb55c..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 @@ -101,17 +96,21 @@ private[spark] class ClientArguments(args: Array[String]) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. val usage = - s""" - |Usage: DriverClient [options] launch [driver options] - |Usage: DriverClient kill - | - |Options: - | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) - | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) - | -s, --supervise Whether to restart the driver on failure - | -v, --verbose Print more debugging output - """.stripMargin + s""" + |Usage: DriverClient [options] launch [driver options] + |Usage: DriverClient kill + | + |Options: + | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) + | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) + | -s, --supervise Whether to restart the driver on failure + | -v, --verbose Print more debugging output + """.stripMargin System.err.println(usage) System.exit(exitCode) } } + +object ClientArguments { + def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar") +} 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 4dfb19ed8adb6..190b331cfe7d8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -1,20 +1,18 @@ /* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * http://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 @@ -23,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 @@ -306,7 +304,8 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - logInfo("Ran %s tests, %s passed and %s failed".format(numPassed+numFailed, numPassed, numFailed)) + logInfo("Ran %s tests, %s passed and %s failed".format(numPassed + numFailed, numPassed, + numFailed)) } private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) 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 e607b8c6f4a36..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} +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) ~ @@ -32,9 +31,12 @@ private[spark] object JsonProtocol { ("webuiaddress" -> obj.webUiAddress) ~ ("cores" -> obj.cores) ~ ("coresused" -> obj.coresUsed) ~ + ("coresfree" -> obj.coresFree) ~ ("memory" -> obj.memory) ~ ("memoryused" -> obj.memoryUsed) ~ - ("state" -> obj.state.toString) + ("memoryfree" -> obj.memoryFree) ~ + ("state" -> obj.state.toString) ~ + ("lastheartbeat" -> obj.lastHeartbeat) } def writeApplicationInfo(obj: ApplicationInfo) = { @@ -54,7 +56,9 @@ private[spark] object JsonProtocol { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ ("memoryperslave" -> obj.memoryPerSlave) ~ - ("user" -> obj.user) + ("user" -> obj.user) ~ + ("sparkhome" -> obj.sparkHome) ~ + ("command" -> obj.command.toString) } def writeExecutorRunner(obj: ExecutorRunner) = { @@ -64,6 +68,14 @@ private[spark] object JsonProtocol { ("appdesc" -> writeApplicationDescription(obj.appDesc)) } + def writeDriverInfo(obj: DriverInfo) = { + ("id" -> obj.id) ~ + ("starttime" -> obj.startTime.toString) ~ + ("state" -> obj.state.toString) ~ + ("cores" -> obj.desc.cores) ~ + ("memory" -> obj.desc.mem) + } + def writeMasterState(obj: MasterStateResponse) = { ("url" -> obj.uri) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ @@ -73,6 +85,7 @@ private[spark] object JsonProtocol { ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~ ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~ ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~ + ("activedrivers" -> obj.activeDrivers.toList.map(writeDriverInfo)) ~ ("status" -> obj.status.toString) } 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 ffc0cb09032fb..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 @@ -33,7 +33,8 @@ import scala.collection.mutable.ArrayBuffer * fault recovery without spinning up a lot of processes. */ private[spark] -class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging { +class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) + extends Logging { private val localHostname = Utils.localHostName() private val masterActorSystems = ArrayBuffer[ActorSystem]() diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 1415e2f3d1886..8901806de9262 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -132,7 +132,8 @@ private[spark] class AppClient( case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => val fullId = appId + "/" + id - logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, + cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index 55d4ef1b31aaa..1f20aa3dfa39b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -33,7 +33,7 @@ private[spark] trait AppClientListener { /** Dead means that we couldn't find any Masters to connect to, and have given up. */ def dead(): Unit - def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit + def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit } 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 2ef167ffc00f0..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() @@ -149,10 +148,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def receive = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() - state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) + state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { RecoveryState.ALIVE - else + } else { RecoveryState.RECOVERING + } logInfo("I have been elected leader! New state: " + state) if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedDrivers, storedWorkers) @@ -165,25 +165,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act System.exit(0) } - case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => { + case RegisterWorker(id, workerHost, workerPort, cores, memory, workerUiPort, publicAddress) => + { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( - host, workerPort, cores, Utils.megabytesToString(memory))) + workerHost, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { // ignore, don't send response } else if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, - sender, workerWebUiPort, publicAddress) + sender, workerUiPort, publicAddress) if (registerWorker(worker)) { persistenceEngine.addWorker(worker) sender ! RegisteredWorker(masterUrl, masterWebUiUrl) schedule() } else { val workerAddress = worker.actor.path.address - logWarning("Worker registration failed. Attempted to re-register worker at same address: " + - workerAddress) - sender ! RegisterWorkerFailed("Attempted to re-register worker at same address: " + workerAddress) + logWarning("Worker registration failed. Attempted to re-register worker at same " + + "address: " + workerAddress) + sender ! RegisterWorkerFailed("Attempted to re-register worker at same address: " + + workerAddress) } } } @@ -641,8 +643,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act worker.id, WORKER_TIMEOUT/1000)) removeWorker(worker) } else { - if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) + if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + } } } } 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 f24f49ea8ad9f..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 @@ -87,7 +87,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = { - val fileData = zk.getData("/spark/master_status/" + filename) + val fileData = zk.getData(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) serializer.fromBinary(fileData).asInstanceOf[T] 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 a9af8df5525d6..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} @@ -57,7 +57,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val completedApps = state.completedApps.sortBy(_.endTime).reverse val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) - val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class") + val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", + "Main Class") val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers) val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse @@ -103,13 +104,14 @@ private[spark] class IndexPage(parent: MasterWebUI) {
- {if (hasDrivers) -
-
-

Running Drivers

- {activeDriversTable} -
-
+ {if (hasDrivers) { +
+
+

Running Drivers

+ {activeDriversTable} +
+
+ } }
@@ -121,13 +123,15 @@ private[spark] class IndexPage(parent: MasterWebUI) {
- {if (hasDrivers) -
-
-

Completed Drivers

- {completedDriversTable} -
-
+ { + if (hasDrivers) { +
+
+

Completed Drivers

+ {completedDriversTable} +
+
+ } }
; @@ -175,7 +179,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { {driver.id} {driver.submitDate} - {driver.worker.map(w => {w.id.toString}).getOrElse("None")} + {driver.worker.map(w => {w.id.toString}).getOrElse("None")} + {driver.state} {driver.desc.cores} 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 ead35662fc75a..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 @@ -45,7 +46,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) server = Some(srv) boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) 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 460883ec7ae24..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 @@ -49,7 +49,8 @@ object CommandUtils extends Logging { val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) .map(p => List("-Djava.library.path=" + p)) .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) + val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) + .map(Utils.splitCommandString).getOrElse(Nil) val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 6f6c101547c3c..a26e47950a0ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -45,4 +45,4 @@ object DriverWrapper { System.exit(-1) } } -} \ No newline at end of file +} 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 old mode 100644 new mode 100755 index fbf2e0f30fde9..7b0b7861b76e1 --- 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} @@ -325,6 +324,7 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) + drivers.values.foreach(_.kill()) webUi.stop() metricsSystem.stop() } 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/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 1dc39c450ea16..530c147000904 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -69,4 +69,4 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor case e => logWarning(s"Received unexpected actor system event: $e") } -} \ No newline at end of file +} 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 925c6fb1832d7..3089acffb8d98 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 @@ -84,7 +84,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {runningExecutorTable} - + // scalastyle:off
{if (hasDrivers)
@@ -113,7 +113,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
}
; - + // scalastyle:on UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format( workerState.host, workerState.port)) } @@ -133,10 +133,10 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr + stdout + stderr 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 8daa47b2b2435..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._ @@ -56,7 +56,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) server = Some(srv) boundPort = Some(bPort) logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) @@ -187,7 +187,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte+logPageLength, logLength) + val endByte = math.min(startByte + logPageLength, logLength) (startByte, endByte) } 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/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index f7efd74e1b043..989d666f15600 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -205,7 +205,7 @@ private[spark] class Executor( } attemptedTask = Some(task) - logDebug("Task " + taskId +"'s epoch is " + task.epoch) + logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) // Run the actual task and measure its runtime. @@ -233,7 +233,8 @@ private[spark] class Executor( val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.getOrElse(null)) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, + task.metrics.getOrElse(null)) val serializedDirectResult = ser.serialize(directResult) logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) val serializedResult = { 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/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index e5c9bbbe2874e..210f3dbeebaca 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -50,10 +50,11 @@ object ExecutorExitCode { "Failed to create local directory (bad spark.local.dir?)" case _ => "Unknown executor exit code (" + exitCode + ")" + ( - if (exitCode > 128) + if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" - else + } else { "" + } ) } } 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 97176e4f5b727..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 { @@ -55,7 +54,8 @@ class ExecutorSource(val executor: Executor, executorId: String) extends Source override def getValue: Int = executor.threadPool.getPoolSize() }) - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + // Gauge got executor thread pool's largest number of threads that have ever simultaneously + // been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "maxPool_size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) 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/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0c8f4662a5f3a..455339943f42d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -64,7 +64,8 @@ class TaskMetrics extends Serializable { var shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected here + * If this task writes to shuffle output, metrics on the written shuffle data will be collected + * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None } 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 9930537b34db0..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 @@ -56,7 +56,8 @@ import org.apache.spark.metrics.source.Source * wild card "*" can be used to replace instance name, which means all the instances will have * this property. * - * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * [sink|source] means this property belongs to source or sink. This field can only be + * source or sink. * * [name] specify the name of sink or source, it is custom defined. * 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 cba8477ed5723..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, @@ -211,7 +208,6 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } return chunk } else { - /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/ message.finishTime = System.currentTimeMillis logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "] in " + message.timeTaken ) @@ -238,7 +234,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, message.startTime = System.currentTimeMillis } logTrace( - "Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") + "Sending chunk from [" + message + "] to [" + getRemoteConnectionManagerId() + "]") return chunk } else { message.finishTime = System.currentTimeMillis @@ -349,8 +345,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, outbox.getChunk() match { case Some(chunk) => { val buffers = chunk.buffers - // If we have 'seen' pending messages, then reset flag - since we handle that as normal - // registering of event (below) + // If we have 'seen' pending messages, then reset flag - since we handle that as + // normal registering of event (below) if (needForceReregister && buffers.exists(_.remaining() > 0)) resetForceReregister() currentBuffers ++= buffers } @@ -404,7 +400,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } catch { case e: Exception => - logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), e) + logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), + e) callOnExceptionCallback(e) close() } 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 e6e01783c8895..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 { @@ -65,7 +62,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) - // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap + // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : + // which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.connect.threads.min", 1), conf.getInt("spark.core.connection.connect.threads.max", 8), @@ -73,8 +71,10 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() - private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] - private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] + private val connectionsByKey = + new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] + private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] + with SynchronizedMap[ConnectionManagerId, SendingConnection] private val messageStatuses = new HashMap[Int, MessageStatus] private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] private val registerRequests = new SynchronizedQueue[SendingConnection] @@ -173,7 +173,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (conn == null) return // prevent other events from being triggered - // Since we are still trying to connect, we do not need to do the additional steps in triggerWrite + // Since we are still trying to connect, we do not need to do the additional steps in + // triggerWrite conn.changeConnectionKeyInterest(0) handleConnectExecutor.execute(new Runnable { @@ -188,8 +189,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } // fallback to previous behavior : we should not really come here since this method was - // triggered since channel became connectable : but at times, the first finishConnect need not - // succeed : hence the loop to retry a few 'times'. + // triggered since channel became connectable : but at times, the first finishConnect need + // not succeed : hence the loop to retry a few 'times'. conn.finishConnect(true) } } ) @@ -258,8 +259,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " } - logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + - "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + logTrace("Changed key for connection to [" + + connection.getRemoteConnectionManagerId() + "] changed from [" + + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") } } } else { @@ -282,7 +284,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi try { selector.select() } catch { - // Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently. + // Explicitly only dealing with CancelledKeyException here since other exceptions + // should be dealt with differently. case e: CancelledKeyException => { // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -310,7 +313,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } if (selectedKeysCount == 0) { - logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") + logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + + " keys") } if (selectorThread.isInterrupted) { logInfo("Selector thread was interrupted!") @@ -341,7 +345,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi throw new CancelledKeyException() } } catch { - // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + // weird, but we saw this happening - even though key.isValid was true, + // key.isAcceptable would throw CancelledKeyException. case e: CancelledKeyException => { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) @@ -458,7 +463,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } def handleConnectionError(connection: Connection, e: Exception) { - logInfo("Handling connection error on connection to " + connection.getRemoteConnectionManagerId()) + logInfo("Handling connection error on connection to " + + connection.getRemoteConnectionManagerId()) removeConnection(connection) } @@ -495,7 +501,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi status } case None => { - throw new Exception("Could not find reference for received ack message " + message.id) + throw new Exception("Could not find reference for received ack message " + + message.id) null } } @@ -517,7 +524,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (ackMessage.isDefined) { if (!ackMessage.get.isInstanceOf[BufferMessage]) { - logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass()) + logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + + ackMessage.get.getClass()) } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { logDebug("Response to " + bufferMessage + " does not have ack id set") ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id @@ -535,14 +543,16 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { - val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) + val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, + connectionManagerId.port) val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId) registerRequests.enqueue(newConnection) newConnection } - // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it useful in our test-env ... - // If we do re-add it, we should consistently use it everywhere I guess ? + // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it + // useful in our test-env ... If we do re-add it, we should consistently use it everywhere I + // guess ? val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) message.senderAddress = id.toSocketAddress() logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") @@ -558,7 +568,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) : Future[Option[Message]] = { val promise = Promise[Option[Message]] - val status = new MessageStatus(message, connectionManagerId, s => promise.success(s.ackMessage)) + val status = new MessageStatus( + message, connectionManagerId, s => promise.success(s.ackMessage)) messageStatuses.synchronized { messageStatuses += ((message.id, status)) } @@ -566,7 +577,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi promise.future } - def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, message: Message): Option[Message] = { + def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, + message: Message): Option[Message] = { Await.result(sendMessageReliably(connectionManagerId, message), Duration.Inf) } @@ -656,7 +668,8 @@ private[spark] object ConnectionManager { val tput = mb * 1000.0 / ms println("--------------------------") println("Started at " + startTime + ", finished at " + finishTime) - println("Sent " + count + " messages of size " + size + " in " + ms + " ms (" + tput + " MB/s)") + println("Sent " + count + " messages of size " + size + " in " + ms + " ms " + + "(" + tput + " MB/s)") println("--------------------------") println() } @@ -667,7 +680,11 @@ private[spark] object ConnectionManager { println("--------------------------") val size = 10 * 1024 * 1024 val count = 10 - val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put(Array.tabulate[Byte](size * (i + 1))(x => x.toByte))) + val buffers = Array.tabulate(count) { i => + val bufferLen = size * (i + 1) + val bufferContent = Array.tabulate[Byte](bufferLen)(x => x.toByte) + ByteBuffer.allocate(bufferLen).put(bufferContent) + } buffers.foreach(_.flip) val mb = buffers.map(_.remaining).reduceLeft(_ + _) / 1024.0 / 1024.0 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 4f5742d29b367..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,27 +17,24 @@ 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]) { - // - the master URL - // - a list slaves to run connectionTest on - //[num of tasks] - the number of parallel tasks to be initiated default is number of slave hosts - //[size of msg in MB (integer)] - the size of messages to be sent in each task, default is 10 - //[count] - how many times to run, default is 3 - //[await time in seconds] : await time (in seconds), default is 600 + // - the master URL - a list slaves to run connectionTest on + // [num of tasks] - the number of parallel tasks to be initiated default is number of slave + // hosts [size of msg in MB (integer)] - the size of messages to be sent in each task, + // default is 10 [count] - how many times to run, default is 3 [await time in seconds] : + // await time (in seconds), default is 600 if (args.length < 2) { - println("Usage: ConnectionManagerTest [num of tasks] [size of msg in MB (integer)] [count] [await time in seconds)] ") + println("Usage: ConnectionManagerTest [num of tasks] " + + "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } @@ -56,7 +53,8 @@ private[spark] object ConnectionManagerTest extends Logging{ val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second - println("Running "+count+" rounds of test: " + "parallel tasks = " + tasknum + ", msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) + println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + + "msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( i => SparkEnv.get.connectionManager.id).collect() println("\nSlave ConnectionManagerIds") @@ -76,18 +74,21 @@ private[spark] object ConnectionManagerTest extends Logging{ buffer.flip val startTime = System.currentTimeMillis - val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map(slaveConnManagerId => { - val bufferMessage = Message.createBufferMessage(buffer.duplicate) - logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") - connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) - }) + val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => + { + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") + connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) + } + } val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime - val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * 1000.0) + " MB/s" + val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * + 1000.0) + " MB/s" logInfo(resultStr) resultStr }).collect() 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 dcbd183c88d09..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 { @@ -52,17 +52,19 @@ private[spark] object SenderTest { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis /*println("Started timer at " + startTime)*/ - val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match { - case Some(response) => + val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) + .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) new String(buffer.array) - case None => "none" - } + }.getOrElse("none") + val finishTime = System.currentTimeMillis val mb = size / 1024.0 / 1024.0 val ms = finishTime - startTime - /*val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms * 1000.0) + " MB/s"*/ - val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr + // val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms + // * 1000.0) + " MB/s" + val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + + (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr println(resultStr) }) } 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 30e578dd93e8d..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 {} @@ -60,7 +63,8 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) checkpointData.get.cpFile = Some(checkpointPath) override def getPreferredLocations(split: Partition): Seq[String] = { - val status = fs.getFileStatus(new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))) + val status = fs.getFileStatus(new Path(checkpointPath, + CheckpointRDD.splitIdToFile(split.index))) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } 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 cefcc3d2d9420..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 @@ -197,9 +198,9 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc // return the next preferredLocation of some partition of the RDD def next(): (String, Partition) = { - if (it.hasNext) + if (it.hasNext) { it.next() - else { + } else { it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning it.next() } @@ -290,8 +291,10 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) - if (prefPart== None) // if no preferred locations, just use basic power of two + if (prefPart.isEmpty) { + // if no preferred locations, just use basic power of two return minPowerOfTwo + } val prefPartActual = prefPart.get 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 688c310ee9caf..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. @@ -37,8 +35,8 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } /** - * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count - * of the RDD's elements in one operation. + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * count of the RDD's elements in one operation. */ def stats(): StatCounter = { self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b)) 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 370061492da91..d29a1a9881cd4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -22,13 +22,14 @@ 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 org.apache.hadoop.conf.Configuration +import com.clearspring.analytics.stream.cardinality.HyperLogLog +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 @@ -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 /** @@ -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.") @@ -617,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]] @@ -705,7 +710,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + - valueClass.getSimpleName+ ")") + valueClass.getSimpleName + ")") val writer = new SparkHadoopWriter(conf) writer.preSetup() 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 09d0a8189d25c..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 @@ -39,7 +40,8 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt override def equals(other: Any): Boolean = other match { - case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && this.slice == that.slice) + case that: ParallelCollectionPartition[_] => + this.rddId == that.rddId && this.slice == that.slice case _ => false } 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 4c625d062eb9b..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,14 +17,15 @@ 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 corresponding partitions - * of parent RDDs. + * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of + * corresponding partitions of parent RDDs. */ private[spark] class PartitionerAwareUnionRDDPartition( 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 629f7074c17c5..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] @@ -45,7 +45,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], - seed: Long = System.nanoTime) + @transient seed: Long = System.nanoTime) extends RDD[U](prev) { override def getPartitions: Array[Partition] = { 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 8010bb68e31dd..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, @@ -775,18 +772,7 @@ abstract class RDD[T: ClassTag]( /** * Return the number of elements in the RDD. */ - def count(): Long = { - sc.runJob(this, (iter: Iterator[T]) => { - // Use a while loop to count the number of elements rather than iter.size because - // iter.size uses a for loop, which is slightly slower in current version of Scala. - var result = 0L - while (iter.hasNext) { - result += 1L - iter.next() - } - result - }).sum - } + def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** * (Experimental) Approximate version of count() that returns a potentially incomplete result @@ -869,6 +855,29 @@ abstract class RDD[T: ClassTag]( aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() } + /** + * Zips this RDD with its element indices. The ordering is first based on the partition index + * and then the ordering of items within each partition. So the first item in the first + * partition gets index 0, and the last item in the last partition receives the largest index. + * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. + * This method needs to trigger a spark job when this RDD contains more than one partitions. + */ + def zipWithIndex(): RDD[(T, Long)] = new ZippedWithIndexRDD(this) + + /** + * Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, + * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method + * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. + */ + def zipWithUniqueId(): RDD[(T, Long)] = { + val n = this.partitions.size.toLong + this.mapPartitionsWithIndex { case (k, iter) => + iter.zipWithIndex.map { case (item, i) => + (item, i * n + k) + } + } + } + /** * Take the first num elements of the RDD. It works by first scanning one partition, and use the * results from that partition to estimate the number of additional partitions needed to satisfy 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 bc688110f4736..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} /** @@ -35,10 +34,10 @@ private[spark] object CheckpointState extends Enumeration { } /** - * This class contains all the information related to RDD checkpointing. Each instance of this class - * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as, - * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations - * of the checkpointed RDD. + * This class contains all the information related to RDD checkpointing. Each instance of this + * class is associated with a RDD. It manages process of checkpointing of the associated RDD, + * as well as, manages the post-checkpoint state by providing the updated partitions, + * iterator and preferred locations of the checkpointed RDD. */ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) extends Logging with Serializable { @@ -97,7 +96,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { throw new SparkException( - "Checkpoint RDD " + newRDD + "("+ newRDD.partitions.size + ") has different " + + "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.size + ") has different " + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") } 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 2d1bd5b4813c9..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, @@ -71,7 +71,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) - logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) + logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + + valueClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { 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 new file mode 100644 index 0000000000000..e2c301603b4a5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rdd + +import scala.reflect.ClassTag + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.util.Utils + +private[spark] +class ZippedWithIndexRDDPartition(val prev: Partition, val startIndex: Long) + extends Partition with Serializable { + override val index: Int = prev.index +} + +/** + * Represents a RDD zipped with its element indices. The ordering is first based on the partition + * index and then the ordering of items within each partition. So the first item in the first + * partition gets index 0, and the last item in the last partition receives the largest index. + * + * @param prev parent RDD + * @tparam T parent RDD item type + */ +private[spark] +class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, Long)](prev) { + + override def getPartitions: Array[Partition] = { + val n = prev.partitions.size + val startIndices: Array[Long] = + if (n == 0) { + Array[Long]() + } else if (n == 1) { + Array(0L) + } else { + prev.context.runJob( + prev, + Utils.getIteratorSize _, + 0 until n - 1, // do not need to count the last partition + false + ).scanLeft(0L)(_ + _) + } + firstParent[T].partitions.map(x => new ZippedWithIndexRDDPartition(x, startIndices(x.index))) + } + + override def getPreferredLocations(split: Partition): Seq[String] = + firstParent[T].preferredLocations(split.asInstanceOf[ZippedWithIndexRDDPartition].prev) + + override def compute(splitIn: Partition, context: TaskContext): Iterator[(T, Long)] = { + val split = splitIn.asInstanceOf[ZippedWithIndexRDDPartition] + firstParent[T].iterator(split.prev, context).zipWithIndex.map { x => + (x._1, split.startIndex + x._2) + } + } +} 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 21d16fabefaa5..dc5b25d845dc2 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} @@ -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 -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleId => - shuffleToMapStage.remove(shuffleId)) - if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + stageToInfos -= stage + 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)) } - 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 @@ -1082,8 +1085,9 @@ class DAGScheduler( case n: NarrowDependency[_] => for (inPart <- n.getParents(partition)) { val locs = getPreferredLocs(n.rdd, inPart) - if (locs != Nil) + if (locs != Nil) { return locs + } } case _ => } 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 cc10cc0849bc7..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,23 +17,23 @@ 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. */ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], - val path: String) extends Logging { + val path: String) extends Logging { var mapreduceInputFormat: Boolean = false var mapredInputFormat: Boolean = false @@ -41,7 +41,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl validate() override def toString: String = { - "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path + "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", " + + "path : " + path } override def hashCode(): Int = { @@ -50,8 +51,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl hashCode } - // Since we are not doing canonicalization of path, this can be wrong : like relative vs absolute path - // .. which is fine, this is best case effort to remove duplicates - right ? + // Since we are not doing canonicalization of path, this can be wrong : like relative vs + // absolute path .. which is fine, this is best case effort to remove duplicates - right ? override def equals(other: Any): Boolean = other match { case that: InputFormatInfo => { // not checking config - that should be fine, right ? @@ -65,22 +66,26 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl logDebug("validate InputFormatInfo : " + inputFormatClazz + ", path " + path) try { - if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom( + inputFormatClazz)) { logDebug("inputformat is from mapreduce package") mapreduceInputFormat = true } - else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom( + inputFormatClazz)) { logDebug("inputformat is from mapred package") mapredInputFormat = true } else { throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + - " is NOT a supported input format ? does not implement either of the supported hadoop api's") + " is NOT a supported input format ? does not implement either of the supported hadoop " + + "api's") } } catch { case e: ClassNotFoundException => { - throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e) + throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + + " cannot be found ?", e) } } } @@ -125,8 +130,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } private def findPreferredLocations(): Set[SplitInfo] = { - logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + - ", inputFormatClazz : " + inputFormatClazz) + logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + + mapredInputFormat + ", inputFormatClazz : " + inputFormatClazz) if (mapreduceInputFormat) { prefLocsFromMapreduceInputFormat() } @@ -150,8 +155,8 @@ object InputFormatInfo { c) Compute rack info for each host and update rack -> count map based on (b). d) Allocate nodes based on (c) e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node - (even if data locality on that is very high) : this is to prevent fragility of job if a single - (or small set of) hosts go down. + (even if data locality on that is very high) : this is to prevent fragility of job if a + single (or small set of) hosts go down. go to (a) until required nodes are allocated. @@ -159,7 +164,8 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] + = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { 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 f8fa5a9f7a590..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 /** @@ -45,10 +45,11 @@ class JobLogger(val user: String, val logDirName: String) String.valueOf(System.currentTimeMillis())) private val logDir = - if (System.getenv("SPARK_LOG_DIR") != null) + if (System.getenv("SPARK_LOG_DIR") != null) { System.getenv("SPARK_LOG_DIR") - else + } else { "/tmp/spark-%s".format(user) + } private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] private val stageIDToJobID = new HashMap[Int, Int] @@ -116,7 +117,7 @@ class JobLogger(val user: String, val logDirName: String) var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) - writeInfo = DATE_FORMAT.format(date) + ": " +info + writeInfo = DATE_FORMAT.format(date) + ": " + info } jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) } @@ -235,7 +236,8 @@ class JobLogger(val user: String, val logDirName: String) * @param stage Root stage of the job * @param indent Indent number before info, default is 0 */ - protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) { + protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) + { val stageInfo = if (stage.isShuffleMap) { "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index c381348a8d424..d94f6ad924260 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -23,4 +23,5 @@ package org.apache.spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) extends JobResult +private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) + extends JobResult 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 28f3ba53b8425..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 { @@ -36,7 +36,8 @@ private[spark] object ResultTask { val metadataCleaner = new MetadataCleaner( MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) - def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { + def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = + { synchronized { val old = serializedInfoCache.get(stageId).orNull if (old != null) { @@ -55,7 +56,8 @@ private[spark] object ResultTask { } } - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { + def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = + { val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() 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 3cf995ea74244..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) @@ -148,6 +148,6 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) } } parentPool.addSchedulable(manager) - logInfo("Added task set " + manager.name + " tasks to pool "+poolName) + logInfo("Added task set " + manager.name + " tasks to pool " + poolName) } } 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 d8e97c3b7c7b0..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 @@ -37,8 +38,8 @@ case class SparkListenerTaskGettingResult( case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null) - extends SparkListenerEvents +case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], + properties: Properties = null) extends SparkListenerEvents case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) extends SparkListenerEvents @@ -99,11 +100,14 @@ class StatsReportListener extends SparkListener with Logging { showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) //shuffle write - showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten}) + showBytesDistribution("shuffle bytes written:", + (_,metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten)) //fetch & io - showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.fetchWaitTime}) - showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead}) + showMillisDistribution("fetch wait time:", + (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime)) + showBytesDistribution("remote bytes read:", + (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead)) showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown @@ -111,8 +115,10 @@ class StatsReportListener extends SparkListener with Logging { val runtimePcts = stageCompleted.stage.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } - showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") - showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") + showDistribution("executor (non-fetch) time pct: ", + Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + showDistribution("fetch wait time pct: ", + Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") } @@ -147,7 +153,8 @@ private[spark] object StatsReportListener extends Logging { logInfo("\t" + quantiles.mkString("\t")) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) + { dOpt.foreach { d => showDistribution(heading, d, formatNumber)} } @@ -156,8 +163,11 @@ private[spark] object StatsReportListener extends Logging { showDistribution(heading, dOpt, f _) } - def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) - (implicit stage: SparkListenerStageCompleted) { + def showDistribution( + heading: String, + format: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Double]) + (implicit stage: SparkListenerStageCompleted) { showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) } @@ -175,7 +185,8 @@ private[spark] object StatsReportListener extends Logging { } def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, (d => StatsReportListener.millisToString(d.toLong)): Double => String) + showDistribution(heading, dOpt, + (d => StatsReportListener.millisToString(d.toLong)): Double => String) } def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) @@ -212,7 +223,7 @@ private object RuntimePercentage { val denom = totalTime.toDouble val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} val fetch = fetchTime.map{_ / denom} - val exec = (metrics.executorRunTime - fetchTime.getOrElse(0l)) / denom + val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 520c0b29e3536..a78b0186b9eab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -63,8 +63,9 @@ private[spark] class Stage( def addOutputLoc(partition: Int, status: MapStatus) { val prevList = outputLocs(partition) outputLocs(partition) = status :: prevList - if (prevList == Nil) + if (prevList == Nil) { numAvailableOutputs += 1 + } } def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c4d1ad5733b4c..8f320e5c7a74b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -29,7 +29,8 @@ import org.apache.spark.executor.TaskMetrics */ class StageInfo( stage: Stage, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = + mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { val stageId = stage.id /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ 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 3c22edd5248f4..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. */ @@ -70,16 +68,17 @@ class TaskInfo( def running: Boolean = !finished def status: String = { - if (running) + if (running) { "RUNNING" - else if (gettingResult) + } else if (gettingResult) { "GET RESULT" - else if (failed) + } else if (failed) { "FAILED" - else if (successful) + } else if (successful) { "SUCCESS" - else + } else { "UNKNOWN" + } } def duration: Long = { 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 9d3e6158266b8..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] @@ -35,7 +36,8 @@ case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Se /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] -class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) +class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], + var metrics: TaskMetrics) extends TaskResult[T] with Externalizable { def this() = this(null.asInstanceOf[ByteBuffer], null, null) 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/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 17b6d97e90e0a..1cdfed1d7005e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -36,7 +36,8 @@ private[spark] trait TaskScheduler { def start(): Unit // Invoked after system has successfully initialized (typically in spark context). - // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc. + // Yarn uses this to bootstrap allocation of resources based on preferred locations, + // wait for slave registerations, etc. def postStartHook() { } // Disconnect from the cluster. 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 3f0ee7a6d48cb..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 @@ -80,7 +78,7 @@ private[spark] class TaskSetManager( var minShare = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString + var name = "TaskSet_" + taskSet.stageId.toString var parent: Pool = null val runningTasksSet = new HashSet[Long] 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 0208388e86680..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} @@ -120,7 +118,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A sender ! true case DisassociatedEvent(_, address, _) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) + addressToExecutorId.get(address).foreach(removeExecutor(_, + "remote Akka client disassociated")) } 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 33aac52051bfc..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 @@ -51,8 +49,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome, - "http://" + sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, + sparkHome, "http://" + sc.ui.appUIAddress) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -84,7 +82,8 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { + override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, + memory: Int) { logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( fullId, hostPort, cores, Utils.megabytesToString(memory))) } 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 c27049bdb5208..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 @@ -136,7 +135,8 @@ private[spark] class CoarseMesosSchedulerBackend( // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d" + ("cd %s*; " + + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } 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 49781485d9f96..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 /** @@ -130,13 +129,8 @@ private[spark] class MesosSchedulerBackend( private def createExecArg(): Array[Byte] = { if (execArgs == null) { val props = new HashMap[String, String] - val iterator = System.getProperties.entrySet.iterator - while (iterator.hasNext) { - val entry = iterator.next - val (key, value) = (entry.getKey.toString, entry.getValue.toString) - if (key.startsWith("spark.")) { - props(key) = value - } + for ((key,value) <- sc.conf.getAll) { + props(key) = value } // Serialize the map as an array of (String, String) pairs execArgs = Utils.serialize(props.toArray) 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 c14cd47556987..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 @@ -60,7 +60,8 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial try { for (regCls <- conf.getOption("spark.kryo.registrator")) { logDebug("Running user registrator: " + regCls) - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + val reg = Class.forName(regCls, true, classLoader).newInstance() + .asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } } catch { 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 9a5e3cb77e1d5..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,16 +22,16 @@ 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 - * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are - * guaranteed to only be called from one thread at a time. + * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual + * serialization and are guaranteed to only be called from one thread at a time. * * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a - * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence. + * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes + * precedence. */ trait Serializer { def newInstance(): SerializerInstance 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 4fa2ab96d9725..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: * @@ -76,9 +73,9 @@ object BlockFetcherIterator { import blockManager._ - private var _remoteBytesRead = 0l - private var _remoteFetchTime = 0l - private var _fetchWaitTime = 0l + private var _remoteBytesRead = 0L + private var _remoteFetchTime = 0L + private var _fetchWaitTime = 0L if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") 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 ed53558566edf..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, @@ -206,8 +203,9 @@ private[spark] class BlockManager( * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. * - * droppedMemorySize exists to account for when block is dropped from memory to disk (so it is still valid). - * This ensures that update in master will compensate for the increase in memory on slave. + * droppedMemorySize exists to account for when block is dropped from memory to disk (so it + * is still valid). This ensures that update in master will compensate for the increase in + * memory on slave. */ def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) { val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) @@ -224,7 +222,8 @@ private[spark] class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = { + private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, + droppedMemorySize: Long = 0L): Boolean = { val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { info.level match { case null => @@ -282,14 +281,15 @@ private[spark] class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { - return diskStore.getBytes(blockId) match { + diskStore.getBytes(blockId) match { case Some(bytes) => Some(bytes) case None => throw new Exception("Block " + blockId + " not found on disk, though it should be") } + } else { + doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } - doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } private def doGetLocal(blockId: BlockId, asValues: Boolean): Option[Any] = { @@ -701,7 +701,8 @@ private[spark] class BlockManager( diskStore.putBytes(blockId, bytes, level) } } - val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val droppedMemorySize = + if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val blockWasRemoved = memoryStore.remove(blockId) if (!blockWasRemoved) { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") 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 2c1a4e2f5d3a1..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} @@ -61,8 +61,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act override def preStart() { if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule( - 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) + timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, + checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) } super.preStart() } @@ -169,8 +169,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { - logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + - (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") + logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId } } 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 365866d1e3397..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 { @@ -57,9 +56,9 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus val diskSpaceUsed = storageStatusList - .flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_ + _) - .getOrElse(0L) + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) diskSpaceUsed / 1024 / 1024 } 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/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index 59329361f320b..dc62b1efaa7d4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -25,7 +25,8 @@ import org.apache.spark._ import org.apache.spark.network._ private[spark] -class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { +class BlockMessageArray(var blockMessages: Seq[BlockMessage]) + extends Seq[BlockMessage] with Logging { def this(bm: BlockMessage) = this(Array(bm)) @@ -65,7 +66,8 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis - logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") + logDebug("Converted block message array from buffer message in " + + (finishTime - startTime) / 1000.0 + " s") this.blockMessages = newBlockMessages } 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 1720007e4e70b..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,23 +17,23 @@ 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] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, blocks: Map[BlockId, BlockStatus]) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_+_).getOrElse(0L) + def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_+_).getOrElse(0L) + rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def memRemaining : Long = maxMem - memUsed() @@ -48,8 +48,9 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, extends Ordered[RDDInfo] { override def toString = { import Utils.bytesToString - "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, - storageLevel.toString, numCachedPartitions, numPartitions, bytesToString(memSize), bytesToString(diskSize)) + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, + numPartitions, bytesToString(memSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -64,7 +65,8 @@ object StorageUtils { /* Returns RDD-level information, compiled from a list of StorageStatus objects */ def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList(storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) + rddInfoFromBlockStatusList( + storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) } /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ @@ -91,7 +93,8 @@ object StorageUtils { sc.persistentRdds.get(rddId).map { r => val rddName = Option(r.name).getOrElse(rddId.toString) val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, + memSize, diskSize) } }.flatten.toArray 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 7211dbc7c6681..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,25 +17,25 @@ 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 - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. + type Responder[T] = HttpServletRequest => T // Conversions from various types of Responder's to jetty Handlers @@ -92,12 +92,15 @@ private[spark] object JettyUtils extends Logging { } /** - * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. + * Attempts to start a Jetty server at the supplied hostName:port which uses the supplied + * handlers. * * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + def startJettyServer(hostName: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) + = { + val handlersToRegister = handlers.map { case(path, handler) => val contextHandler = new ContextHandler(path) contextHandler.setHandler(handler) @@ -109,7 +112,7 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { - val server = new Server(currentPort) + val server = new Server(new InetSocketAddress(hostName, currentPort)) val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) 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 50dfdbdf5ae9b..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 */ @@ -54,7 +52,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) server = Some(srv) boundPort = Some(usedPort) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 5573b3847bcaf..547a194d58a5c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -48,14 +48,16 @@ private[spark] object UIUtils { case _ =>
  • Environment
  • } val executors = page match { - case Executors =>
  • Executors
  • + case Executors => +
  • Executors
  • case _ =>
  • Executors
  • } - + {sc.appName} - {title} @@ -63,7 +65,9 @@ private[spark] object UIUtils {