diff options
Diffstat (limited to 'core')
158 files changed, 3026 insertions, 1498 deletions
diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index d764ffc29d..c157cc8feb 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -3,6 +3,7 @@ package spark import java.io._ import scala.collection.mutable.Map +import scala.collection.generic.Growable /** * A datatype that can be accumulated, i.e. has an commutative and associative +. @@ -92,6 +93,29 @@ trait AccumulableParam[R, T] extends Serializable { def zero(initialValue: R): R } +class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable, T] + extends AccumulableParam[R,T] { + + def addAccumulator(growable: R, elem: T) : R = { + growable += elem + growable + } + + def addInPlace(t1: R, t2: R) : R = { + t1 ++= t2 + t1 + } + + def zero(initialValue: R): R = { + // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. + // Instead we'll serialize it to a buffer and load it back. + val ser = (new spark.JavaSerializer).newInstance() + val copy = ser.deserialize[R](ser.serialize(initialValue)) + copy.clear() // In case it contained stuff + copy + } +} + /** * A simpler value of [[spark.Accumulable]] where the result type being accumulated is the same * as the types of elements being merged. diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index 6516bea157..b0daa70cfd 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -9,9 +9,9 @@ package spark * known as map-side aggregations. When set to false, * mergeCombiners function is not used. */ -class Aggregator[K, V, C] ( +case class Aggregator[K, V, C] ( val createCombiner: V => C, val mergeValue: (C, V) => C, val mergeCombiners: (C, C) => C, val mapSideCombine: Boolean = true) - extends Serializable + diff --git a/core/src/main/scala/spark/BlockRDD.scala b/core/src/main/scala/spark/BlockRDD.scala index ea009f0f4f..faa99fe3e9 100644 --- a/core/src/main/scala/spark/BlockRDD.scala +++ b/core/src/main/scala/spark/BlockRDD.scala @@ -2,12 +2,13 @@ package spark import scala.collection.mutable.HashMap -class BlockRDDSplit(val blockId: String, idx: Int) extends Split { +private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split { val index = idx } -class BlockRDD[T: ClassManifest](sc: SparkContext, blockIds: Array[String]) extends RDD[T](sc) { +class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) + extends RDD[T](sc) { @transient val splits_ = (0 until blockIds.size).map(i => { diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 45a14c8290..fb65ba421a 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -11,8 +11,7 @@ import spark.storage.BlockManagerId import it.unimi.dsi.fastutil.io.FastBufferedInputStream - -class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { +private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { def fetch[K, V](shuffleId: Int, reduceId: Int, func: (K, V) => Unit) { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -29,39 +28,32 @@ class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { val blocksByAddress: Seq[(BlockManagerId, Seq[String])] = splitsByAddress.toSeq.map { case (address, splits) => - (address, splits.map(i => "shuffleid_%d_%d_%d".format(shuffleId, i, reduceId))) + (address, splits.map(i => "shuffle_%d_%d_%d".format(shuffleId, i, reduceId))) } - try { - for ((blockId, blockOption) <- blockManager.getMultiple(blocksByAddress)) { - blockOption match { - case Some(block) => { - val values = block - for(value <- values) { - val v = value.asInstanceOf[(K, V)] - func(v._1, v._2) - } - } - case None => { - throw new BlockException(blockId, "Did not get block " + blockId) + for ((blockId, blockOption) <- blockManager.getMultiple(blocksByAddress)) { + blockOption match { + case Some(block) => { + val values = block + for(value <- values) { + val v = value.asInstanceOf[(K, V)] + func(v._1, v._2) } } - } - } catch { - // TODO: this is really ugly -- let's find a better way of throwing a FetchFailedException - case be: BlockException => { - val regex = "shuffleid_([0-9]*)_([0-9]*)_([0-9]]*)".r - be.blockId match { - case regex(sId, mId, rId) => { - val address = addresses(mId.toInt) - throw new FetchFailedException(address, sId.toInt, mId.toInt, rId.toInt, be) - } - case _ => { - throw be + case None => { + val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r + blockId match { + case regex(shufId, mapId, reduceId) => + val addr = addresses(mapId.toInt) + throw new FetchFailedException(addr, shufId.toInt, mapId.toInt, reduceId.toInt, null) + case _ => + throw new SparkException( + "Failed to get block " + blockId + ", which is not a shuffle block") } } } } + logDebug("Fetching and merging outputs of shuffle %d, reduce %d took %d ms".format( shuffleId, reduceId, System.currentTimeMillis - startTime)) } diff --git a/core/src/main/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala index 6fe0b94297..e8392a194f 100644 --- a/core/src/main/scala/spark/BoundedMemoryCache.scala +++ b/core/src/main/scala/spark/BoundedMemoryCache.scala @@ -9,7 +9,7 @@ import java.util.LinkedHashMap * some cache entries have pointers to a shared object. Nonetheless, this Cache should work well * when most of the space is used by arrays of primitives or of simple classes. */ -class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging { +private[spark] class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging { logInfo("BoundedMemoryCache.maxBytes = " + maxBytes) def this() { @@ -104,9 +104,9 @@ class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging { } // An entry in our map; stores a cached object and its size in bytes -case class Entry(value: Any, size: Long) +private[spark] case class Entry(value: Any, size: Long) -object BoundedMemoryCache { +private[spark] object BoundedMemoryCache { /** * Get maximum cache capacity from system configuration */ diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala index 150fe14e2c..20d677a854 100644 --- a/core/src/main/scala/spark/Cache.scala +++ b/core/src/main/scala/spark/Cache.scala @@ -2,9 +2,9 @@ package spark import java.util.concurrent.atomic.AtomicInteger -sealed trait CachePutResponse -case class CachePutSuccess(size: Long) extends CachePutResponse -case class CachePutFailure() extends CachePutResponse +private[spark] sealed trait CachePutResponse +private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse +private[spark] case class CachePutFailure() extends CachePutResponse /** * An interface for caches in Spark, to allow for multiple implementations. Caches are used to store @@ -22,7 +22,7 @@ case class CachePutFailure() extends CachePutResponse * This abstract class handles the creation of key spaces, so that subclasses need only deal with * keys that are unique across modules. */ -abstract class Cache { +private[spark] abstract class Cache { private val nextKeySpaceId = new AtomicInteger(0) private def newKeySpaceId() = nextKeySpaceId.getAndIncrement() @@ -52,7 +52,7 @@ abstract class Cache { /** * A key namespace in a Cache. */ -class KeySpace(cache: Cache, val keySpaceId: Int) { +private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) { def get(datasetId: Any, partition: Int): Any = cache.get((keySpaceId, datasetId), partition) diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index 22110832f8..9a23f9e7cc 100644 --- a/core/src/main/scala/spark/CacheTracker.scala +++ b/core/src/main/scala/spark/CacheTracker.scala @@ -15,19 +15,20 @@ import scala.collection.mutable.HashSet import spark.storage.BlockManager import spark.storage.StorageLevel -sealed trait CacheTrackerMessage -case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L) +private[spark] sealed trait CacheTrackerMessage + +private[spark] case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L) extends CacheTrackerMessage -case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L) +private[spark] case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L) extends CacheTrackerMessage -case class MemoryCacheLost(host: String) extends CacheTrackerMessage -case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage -case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage -case object GetCacheStatus extends CacheTrackerMessage -case object GetCacheLocations extends CacheTrackerMessage -case object StopCacheTracker extends CacheTrackerMessage - -class CacheTrackerActor extends Actor with Logging { +private[spark] case class MemoryCacheLost(host: String) extends CacheTrackerMessage +private[spark] case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage +private[spark] case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage +private[spark] case object GetCacheStatus extends CacheTrackerMessage +private[spark] case object GetCacheLocations extends CacheTrackerMessage +private[spark] case object StopCacheTracker extends CacheTrackerMessage + +private[spark] class CacheTrackerActor extends Actor with Logging { // TODO: Should probably store (String, CacheType) tuples private val locs = new HashMap[Int, Array[List[String]]] @@ -43,8 +44,6 @@ class CacheTrackerActor extends Actor with Logging { def receive = { case SlaveCacheStarted(host: String, size: Long) => - logInfo("Started slave cache (size %s) on %s".format( - Utils.memoryBytesToString(size), host)) slaveCapacity.put(host, size) slaveUsage.put(host, 0) sender ! true @@ -56,22 +55,12 @@ class CacheTrackerActor extends Actor with Logging { case AddedToCache(rddId, partition, host, size) => slaveUsage.put(host, getCacheUsage(host) + size) - logInfo("Cache entry added: (%s, %s) on %s (size added: %s, available: %s)".format( - rddId, partition, host, Utils.memoryBytesToString(size), - Utils.memoryBytesToString(getCacheAvailable(host)))) locs(rddId)(partition) = host :: locs(rddId)(partition) sender ! true case DroppedFromCache(rddId, partition, host, size) => - logInfo("Cache entry removed: (%s, %s) on %s (size dropped: %s, available: %s)".format( - rddId, partition, host, Utils.memoryBytesToString(size), - Utils.memoryBytesToString(getCacheAvailable(host)))) slaveUsage.put(host, getCacheUsage(host) - size) // Do a sanity check to make sure usage is greater than 0. - val usage = getCacheUsage(host) - if (usage < 0) { - logError("Cache usage on %s is negative (%d)".format(host, usage)) - } locs(rddId)(partition) = locs(rddId)(partition).filterNot(_ == host) sender ! true @@ -101,7 +90,7 @@ class CacheTrackerActor extends Actor with Logging { } } -class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager) +private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager) extends Logging { // Tracker actor on the master, or remote reference to it on workers @@ -151,7 +140,6 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl logInfo("Registering RDD ID " + rddId + " with cache") registeredRddIds += rddId communicate(RegisterRDD(rddId, numPartitions)) - logInfo(RegisterRDD(rddId, numPartitions) + " successful") } } } @@ -171,7 +159,6 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl // For BlockManager.scala only def notifyTheCacheTrackerFromBlockManager(t: AddedToCache) { communicate(t) - logInfo("notifyTheCacheTrackerFromBlockManager successful") } // Get a snapshot of the currently known locations @@ -181,7 +168,7 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl // Gets or computes an RDD split def getOrCompute[T](rdd: RDD[T], split: Split, storageLevel: StorageLevel): Iterator[T] = { - val key = "rdd:%d:%d".format(rdd.id, split.index) + val key = "rdd_%d_%d".format(rdd.id, split.index) logInfo("Cache key is " + key) blockManager.get(key) match { case Some(cachedValues) => @@ -223,7 +210,7 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl logInfo("Computing partition " + split) try { // BlockManager will iterate over results from compute to create RDD - blockManager.put(key, rdd.compute(split), storageLevel, false) + blockManager.put(key, rdd.compute(split), storageLevel, true) //future.apply() // Wait for the reply from the cache tracker blockManager.get(key) match { case Some(values) => diff --git a/core/src/main/scala/spark/CartesianRDD.scala b/core/src/main/scala/spark/CartesianRDD.scala index e26041555a..83db2d2934 100644 --- a/core/src/main/scala/spark/CartesianRDD.scala +++ b/core/src/main/scala/spark/CartesianRDD.scala @@ -1,5 +1,6 @@ package spark +private[spark] class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable { override val index: Int = idx } diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index 3b83d23a13..98525b99c8 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -9,7 +9,7 @@ import org.objectweb.asm.{ClassReader, MethodVisitor, Type} import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ -object ClosureCleaner extends Logging { +private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it private def getClassReader(cls: Class[_]): ClassReader = { new ClassReader(cls.getResourceAsStream( @@ -154,7 +154,7 @@ object ClosureCleaner extends Logging { } } -class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor { +private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { return new EmptyVisitor { @@ -180,7 +180,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor } } -class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor { +private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor { var myName: String = null override def visit(version: Int, access: Int, name: String, sig: String, diff --git a/core/src/main/scala/spark/CoGroupedRDD.scala b/core/src/main/scala/spark/CoGroupedRDD.scala index 6959917d14..daba719b14 100644 --- a/core/src/main/scala/spark/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/CoGroupedRDD.scala @@ -6,16 +6,17 @@ import java.io.ObjectInputStream import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -sealed trait CoGroupSplitDep extends Serializable -case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep -case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep +private[spark] sealed trait CoGroupSplitDep extends Serializable +private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep +private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep +private[spark] class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable { override val index: Int = idx override def hashCode(): Int = idx } -class CoGroupAggregator +private[spark] class CoGroupAggregator extends Aggregator[Any, Any, ArrayBuffer[Any]]( { x => ArrayBuffer(x) }, { (b, x) => b += x }, diff --git a/core/src/main/scala/spark/CoalescedRDD.scala b/core/src/main/scala/spark/CoalescedRDD.scala new file mode 100644 index 0000000000..f1ae346a44 --- /dev/null +++ b/core/src/main/scala/spark/CoalescedRDD.scala @@ -0,0 +1,43 @@ +package spark + +private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split + +/** + * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of + * this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the + * parent had more than this many partitions, or fewer if the parent had fewer. + * + * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, + * or to avoid having a large number of small tasks when processing a directory with many files. + */ +class CoalescedRDD[T: ClassManifest](prev: RDD[T], maxPartitions: Int) + extends RDD[T](prev.context) { + + @transient val splits_ : Array[Split] = { + val prevSplits = prev.splits + if (prevSplits.length < maxPartitions) { + prevSplits.zipWithIndex.map{ case (s, idx) => new CoalescedRDDSplit(idx, Array(s)) } + } else { + (0 until maxPartitions).map { i => + val rangeStart = (i * prevSplits.length) / maxPartitions + val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions + new CoalescedRDDSplit(i, prevSplits.slice(rangeStart, rangeEnd)) + }.toArray + } + } + + override def splits = splits_ + + override def compute(split: Split): Iterator[T] = { + split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { + parentSplit => prev.iterator(parentSplit) + } + } + + val dependencies = List( + new NarrowDependency(prev) { + def getParents(id: Int): Seq[Int] = + splits(id).asInstanceOf[CoalescedRDDSplit].parents.map(_.index) + } + ) +} diff --git a/core/src/main/scala/spark/DaemonThreadFactory.scala b/core/src/main/scala/spark/DaemonThreadFactory.scala index 003880c5e8..56e59adeb7 100644 --- a/core/src/main/scala/spark/DaemonThreadFactory.scala +++ b/core/src/main/scala/spark/DaemonThreadFactory.scala @@ -6,9 +6,13 @@ import java.util.concurrent.ThreadFactory * A ThreadFactory that creates daemon threads */ private object DaemonThreadFactory extends ThreadFactory { - override def newThread(r: Runnable): Thread = { - val t = new Thread(r) - t.setDaemon(true) - return t + override def newThread(r: Runnable): Thread = new DaemonThread(r) +} + +private class DaemonThread(r: Runnable = null) extends Thread { + override def run() { + if (r != null) { + r.run() + } } }
\ No newline at end of file diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala index 55512f4481..a953081d24 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/spark/FetchFailedException.scala @@ -2,7 +2,7 @@ package spark import spark.storage.BlockManagerId -class FetchFailedException( +private[spark] class FetchFailedException( val bmAddress: BlockManagerId, val shuffleId: Int, val mapId: Int, diff --git a/core/src/main/scala/spark/HadoopRDD.scala b/core/src/main/scala/spark/HadoopRDD.scala index f282a4023b..6d448116a9 100644 --- a/core/src/main/scala/spark/HadoopRDD.scala +++ b/core/src/main/scala/spark/HadoopRDD.scala @@ -18,7 +18,7 @@ import org.apache.hadoop.util.ReflectionUtils /** * A Spark split class that wraps around a Hadoop InputSplit. */ -class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit) +private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit) extends Split with Serializable { @@ -42,7 +42,8 @@ class HadoopRDD[K, V]( minSplits: Int) extends RDD[(K, V)](sc) { - val serializableConf = new SerializableWritable(conf) + // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it + val confBroadcast = sc.broadcast(new SerializableWritable(conf)) @transient val splits_ : Array[Split] = { @@ -66,7 +67,7 @@ class HadoopRDD[K, V]( val split = theSplit.asInstanceOf[HadoopSplit] var reader: RecordReader[K, V] = null - val conf = serializableConf.value + val conf = confBroadcast.value.value val fmt = createInputFormat(conf) reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala new file mode 100644 index 0000000000..659d17718f --- /dev/null +++ b/core/src/main/scala/spark/HttpFileServer.scala @@ -0,0 +1,47 @@ +package spark + +import java.io.{File, PrintWriter} +import java.net.URL +import scala.collection.mutable.HashMap +import org.apache.hadoop.fs.FileUtil + +private[spark] class HttpFileServer extends Logging { + + var baseDir : File = null + var fileDir : File = null + var jarDir : File = null + var httpServer : HttpServer = null + var serverUri : String = null + + def initialize() { + baseDir = Utils.createTempDir() + fileDir = new File(baseDir, "files") + jarDir = new File(baseDir, "jars") + fileDir.mkdir() + jarDir.mkdir() + logInfo("HTTP File server directory is " + baseDir) + httpServer = new HttpServer(baseDir) + httpServer.start() + serverUri = httpServer.uri + } + + def stop() { + httpServer.stop() + } + + def addFile(file: File) : String = { + addFileToDir(file, fileDir) + return serverUri + "/files/" + file.getName + } + + def addJar(file: File) : String = { + addFileToDir(file, jarDir) + return serverUri + "/jars/" + file.getName + } + + def addFileToDir(file: File, dir: File) : String = { + Utils.copyFile(file, new File(dir, file.getName)) + return dir + "/" + file.getName + } + +}
\ No newline at end of file diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala index 855f2c752f..0196595ba1 100644 --- a/core/src/main/scala/spark/HttpServer.scala +++ b/core/src/main/scala/spark/HttpServer.scala @@ -12,14 +12,14 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool /** * Exception type thrown by HttpServer when it is in the wrong state for an operation. */ -class ServerStateException(message: String) extends Exception(message) +private[spark] class ServerStateException(message: String) extends Exception(message) /** * An HTTP server for static content used to allow worker nodes to access JARs added to SparkContext * as well as classes created by the interpreter when the user types in code. This is just a wrapper * around a Jetty server. */ -class HttpServer(resourceBase: File) extends Logging { +private[spark] class HttpServer(resourceBase: File) extends Logging { private var server: Server = null private var port: Int = -1 diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala index d11ba5167d..39d554b6a5 100644 --- a/core/src/main/scala/spark/JavaSerializer.scala +++ b/core/src/main/scala/spark/JavaSerializer.scala @@ -5,14 +5,14 @@ import java.nio.ByteBuffer import spark.util.ByteBufferInputStream -class JavaSerializationStream(out: OutputStream) extends SerializationStream { +private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { val objOut = new ObjectOutputStream(out) - def writeObject[T](t: T) { objOut.writeObject(t) } + def writeObject[T](t: T): SerializationStream = { objOut.writeObject(t); this } def flush() { objOut.flush() } def close() { objOut.close() } } -class JavaDeserializationStream(in: InputStream, loader: ClassLoader) +private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) extends DeserializationStream { val objIn = new ObjectInputStream(in) { override def resolveClass(desc: ObjectStreamClass) = @@ -23,7 +23,7 @@ extends DeserializationStream { def close() { objIn.close() } } -class JavaSerializerInstance extends SerializerInstance { +private[spark] class JavaSerializerInstance extends SerializerInstance { def serialize[T](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -57,6 +57,6 @@ class JavaSerializerInstance extends SerializerInstance { } } -class JavaSerializer extends Serializer { +private[spark] class JavaSerializer extends Serializer { def newInstance(): SerializerInstance = new JavaSerializerInstance } diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 65d0532bd5..b8aa3a86c5 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -10,15 +10,17 @@ import scala.collection.mutable import com.esotericsoftware.kryo._ import com.esotericsoftware.kryo.{Serializer => KSerializer} import com.esotericsoftware.kryo.serialize.ClassSerializer +import com.esotericsoftware.kryo.serialize.SerializableSerializer import de.javakaffee.kryoserializers.KryoReflectionFactorySupport +import spark.broadcast._ import spark.storage._ /** * Zig-zag encoder used to write object sizes to serialization streams. * Based on Kryo's integer encoder. */ -object ZigZag { +private[spark] object ZigZag { def writeInt(n: Int, out: OutputStream) { var value = n if ((value & ~0x7F) == 0) { @@ -66,22 +68,25 @@ object ZigZag { } } +private[spark] class KryoSerializationStream(kryo: Kryo, threadBuffer: ByteBuffer, out: OutputStream) extends SerializationStream { val channel = Channels.newChannel(out) - def writeObject[T](t: T) { + def writeObject[T](t: T): SerializationStream = { kryo.writeClassAndObject(threadBuffer, t) ZigZag.writeInt(threadBuffer.position(), out) threadBuffer.flip() channel.write(threadBuffer) threadBuffer.clear() + this } def flush() { out.flush() } def close() { out.close() } } +private[spark] class KryoDeserializationStream(objectBuffer: ObjectBuffer, in: InputStream) extends DeserializationStream { def readObject[T](): T = { @@ -92,7 +97,7 @@ extends DeserializationStream { def close() { in.close() } } -class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { +private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { val kryo = ks.kryo val threadBuffer = ks.threadBuffer.get() val objectBuffer = ks.objectBuffer.get() @@ -159,7 +164,9 @@ trait KryoRegistrator { } class KryoSerializer extends Serializer with Logging { - val kryo = createKryo() + // Make this lazy so that it only gets called once we receive our first task on each executor, + // so we can pull out any custom Kryo registrator from the user's JARs. + lazy val kryo = createKryo() val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "32").toInt * 1024 * 1024 @@ -190,8 +197,8 @@ class KryoSerializer extends Serializer with Logging { (1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1), None, ByteBuffer.allocate(1), - StorageLevel.MEMORY_ONLY_DESER, - PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY_DESER), + StorageLevel.MEMORY_ONLY, + PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), GotBlock("1", ByteBuffer.allocate(1)), GetBlock("1") ) @@ -203,6 +210,10 @@ class KryoSerializer extends Serializer with Logging { kryo.register(classOf[Class[_]], new ClassSerializer(kryo)) kryo.setRegistrationOptional(true) + // Allow sending SerializableWritable + kryo.register(classOf[SerializableWritable[_]], new SerializableSerializer()) + kryo.register(classOf[HttpBroadcast[_]], new SerializableSerializer()) + // Register some commonly used Scala singleton objects. Because these // are singletons, we must return the exact same local object when we // deserialize rather than returning a clone as FieldSerializer would. @@ -250,7 +261,8 @@ class KryoSerializer extends Serializer with Logging { val regCls = System.getProperty("spark.kryo.registrator") if (regCls != null) { logInfo("Running user registrator: " + regCls) - val reg = Class.forName(regCls).newInstance().asInstanceOf[KryoRegistrator] + val classLoader = Thread.currentThread.getContextClassLoader + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } kryo diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index de23eb6f48..116d526854 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -1,5 +1,6 @@ package spark +import java.io.{DataInputStream, DataOutputStream, ByteArrayOutputStream, ByteArrayInputStream} import java.util.concurrent.ConcurrentHashMap import akka.actor._ @@ -10,20 +11,20 @@ import akka.util.Duration import akka.util.Timeout import akka.util.duration._ +import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import spark.storage.BlockManagerId -sealed trait MapOutputTrackerMessage -case class GetMapOutputLocations(shuffleId: Int) extends MapOutputTrackerMessage -case object StopMapOutputTracker extends MapOutputTrackerMessage +private[spark] sealed trait MapOutputTrackerMessage +private[spark] case class GetMapOutputLocations(shuffleId: Int) extends MapOutputTrackerMessage +private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -class MapOutputTrackerActor(bmAddresses: ConcurrentHashMap[Int, Array[BlockManagerId]]) -extends Actor with Logging { +private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Logging { def receive = { case GetMapOutputLocations(shuffleId: Int) => logInfo("Asked to get map output locations for shuffle " + shuffleId) - sender ! bmAddresses.get(shuffleId) + sender ! tracker.getSerializedLocations(shuffleId) case StopMapOutputTracker => logInfo("MapOutputTrackerActor stopped!") @@ -32,22 +33,26 @@ extends Actor with Logging { } } -class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging { +private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging { val ip: String = System.getProperty("spark.master.host", "localhost") val port: Int = System.getProperty("spark.master.port", "7077").toInt val actorName: String = "MapOutputTracker" val timeout = 10.seconds - private var bmAddresses = new ConcurrentHashMap[Int, Array[BlockManagerId]] + var bmAddresses = new ConcurrentHashMap[Int, Array[BlockManagerId]] // Incremented every time a fetch fails so that client nodes know to clear // their cache of map output locations if this happens. private var generation: Long = 0 private var generationLock = new java.lang.Object + // Cache a serialized version of the output locations for each shuffle to send them out faster + var cacheGeneration = generation + val cachedSerializedLocs = new HashMap[Int, Array[Byte]] + var trackerActor: ActorRef = if (isMaster) { - val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(bmAddresses)), name = actorName) + val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName) logInfo("Registered MapOutputTrackerActor actor") actor } else { @@ -134,15 +139,16 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg } // We won the race to fetch the output locs; do so logInfo("Doing the fetch; tracker actor = " + trackerActor) - val fetched = askTracker(GetMapOutputLocations(shuffleId)).asInstanceOf[Array[BlockManagerId]] + val fetchedBytes = askTracker(GetMapOutputLocations(shuffleId)).asInstanceOf[Array[Byte]] + val fetchedLocs = deserializeLocations(fetchedBytes) logInfo("Got the output locations") - bmAddresses.put(shuffleId, fetched) + bmAddresses.put(shuffleId, fetchedLocs) fetching.synchronized { fetching -= shuffleId fetching.notifyAll() } - return fetched + return fetchedLocs } else { return locs } @@ -181,4 +187,70 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg } } } + + def getSerializedLocations(shuffleId: Int): Array[Byte] = { + var locs: Array[BlockManagerId] = null + var generationGotten: Long = -1 + generationLock.synchronized { + if (generation > cacheGeneration) { + cachedSerializedLocs.clear() + cacheGeneration = generation + } + cachedSerializedLocs.get(shuffleId) match { + case Some(bytes) => + return bytes + case None => + locs = bmAddresses.get(shuffleId) + generationGotten = generation + } + } + // If we got here, we failed to find the serialized locations in the cache, so we pulled + // out a snapshot of the locations as "locs"; let's serialize and return that + val bytes = serializeLocations(locs) + // Add them into the table only if the generation hasn't changed while we were working + generationLock.synchronized { + if (generation == generationGotten) { + cachedSerializedLocs(shuffleId) = bytes + } + } + return bytes + } + + // Serialize an array of map output locations into an efficient byte format so that we can send + // it to reduce tasks. We do this by grouping together the locations by block manager ID. + def serializeLocations(locs: Array[BlockManagerId]): Array[Byte] = { + val out = new ByteArrayOutputStream + val dataOut = new DataOutputStream(out) + dataOut.writeInt(locs.length) + val grouped = locs.zipWithIndex.groupBy(_._1) + dataOut.writeInt(grouped.size) + for ((id, pairs) <- grouped if id != null) { + dataOut.writeUTF(id.ip) + dataOut.writeInt(id.port) + dataOut.writeInt(pairs.length) + for ((_, blockIndex) <- pairs) { + dataOut.writeInt(blockIndex) + } + } + dataOut.close() + out.toByteArray + } + + // Opposite of serializeLocations. + def deserializeLocations(bytes: Array[Byte]): Array[BlockManagerId] = { + val dataIn = new DataInputStream(new ByteArrayInputStream(bytes)) + val length = dataIn.readInt() + val array = new Array[BlockManagerId](length) + val numGroups = dataIn.readInt() + for (i <- 0 until numGroups) { + val ip = dataIn.readUTF() + val port = dataIn.readInt() + val id = new BlockManagerId(ip, port) + val numBlocks = dataIn.readInt() + for (j <- 0 until numBlocks) { + array(dataIn.readInt()) = id + } + } + array + } } diff --git a/core/src/main/scala/spark/NewHadoopRDD.scala b/core/src/main/scala/spark/NewHadoopRDD.scala index d024d38aa9..9072698357 100644 --- a/core/src/main/scala/spark/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/NewHadoopRDD.scala @@ -13,6 +13,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptID import java.util.Date import java.text.SimpleDateFormat +private[spark] class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) extends Split { @@ -28,7 +29,9 @@ class NewHadoopRDD[K, V]( @transient conf: Configuration) extends RDD[(K, V)](sc) { - private val serializableConf = new SerializableWritable(conf) + // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it + val confBroadcast = sc.broadcast(new SerializableWritable(conf)) + // private val serializableConf = new SerializableWritable(conf) private val jobtrackerId: String = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") @@ -41,7 +44,7 @@ class NewHadoopRDD[K, V]( @transient private val splits_ : Array[Split] = { val inputFormat = inputFormatClass.newInstance - val jobContext = new JobContext(serializableConf.value, jobId) + val jobContext = new JobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Split](rawSplits.size) for (i <- 0 until rawSplits.size) { @@ -54,9 +57,9 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Split) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopSplit] - val conf = serializableConf.value + val conf = confBroadcast.value.value val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) - val context = new TaskAttemptContext(serializableConf.value, attemptId) + val context = new TaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance val reader = format.createRecordReader(split.serializableHadoopSplit.value, context) reader.initialize(split.serializableHadoopSplit.value, context) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 64018f8c6b..80d62caf25 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,11 +1,10 @@ package spark import java.io.EOFException -import java.net.URL import java.io.ObjectInputStream +import java.net.URL +import java.util.{Date, HashMap => JHashMap} import java.util.concurrent.atomic.AtomicLong -import java.util.{HashMap => JHashMap} -import java.util.Date import java.text.SimpleDateFormat import scala.collection.Map @@ -50,9 +49,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - partitioner: Partitioner): RDD[(K, C)] = { - val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) - new ShuffledRDD(self, aggregator, partitioner) + partitioner: Partitioner, + mapSideCombine: Boolean = true): RDD[(K, C)] = { + val aggregator = + if (mapSideCombine) { + new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) + } else { + // Don't apply map-side combiner. + // A sanity check to make sure mergeCombiners is not defined. + assert(mergeCombiners == null) + new Aggregator[K, V, C](createCombiner, mergeValue, null, false) + } + new ShuffledAggregatedRDD(self, aggregator, partitioner) } def combineByKey[C](createCombiner: V => C, @@ -65,7 +73,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = { combineByKey[V]((v: V) => v, func, func, partitioner) } - + def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { val map = new JHashMap[K, V] @@ -116,13 +124,24 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( groupByKey(new HashPartitioner(numSplits)) } - def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - def createCombiner(v: V) = ArrayBuffer(v) - def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2 - val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, mergeCombiners _, partitioner) - bufs.flatMapValues(buf => buf) + /** + * Repartition the RDD using the specified partitioner. If mapSideCombine is + * true, Spark will group values of the same key together on the map side + * before the repartitioning. If a large number of duplicated keys are + * expected, and the size of the keys are large, mapSideCombine should be set + * to true. + */ + def partitionBy(partitioner: Partitioner, mapSideCombine: Boolean = false): RDD[(K, V)] = { + if (mapSideCombine) { + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2 + val bufs = combineByKey[ArrayBuffer[V]]( + createCombiner _, mergeValue _, mergeCombiners _, partitioner) + bufs.flatMapValues(buf => buf) + } else { + new RepartitionShuffledRDD(self, partitioner) + } } def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { @@ -194,17 +213,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } def collectAsMap(): Map[K, V] = HashMap(self.collect(): _*) - + def mapValues[U](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } - + def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } - + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { val cg = new CoGroupedRDD[K]( Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]), @@ -215,12 +234,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) } } - + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { val cg = new CoGroupedRDD[K]( Seq(self.asInstanceOf[RDD[(_, _)]], - other1.asInstanceOf[RDD[(_, _)]], + other1.asInstanceOf[RDD[(_, _)]], other2.asInstanceOf[RDD[(_, _)]]), partitioner) val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) @@ -289,7 +308,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } - + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -363,7 +382,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) saveAsHadoopDataset(conf) } - + def saveAsHadoopDataset(conf: JobConf) { val outputFormatClass = conf.getOutputFormat val keyClass = conf.getOutputKeyClass @@ -377,7 +396,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( if (valueClass == null) { throw new SparkException("Output value class not set") } - + logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")") val writer = new HadoopWriter(conf) @@ -390,14 +409,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( writer.setup(context.stageId, context.splitId, attemptNumber) writer.open() - + var count = 0 while(iter.hasNext) { val record = iter.next count += 1 writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) } - + writer.close() writer.commit() } @@ -413,28 +432,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( self: RDD[(K, V)]) - extends Logging + extends Logging with Serializable { - def sortByKey(ascending: Boolean = true): RDD[(K,V)] = { - val rangePartitionedRDD = self.partitionBy(new RangePartitioner(self.splits.size, self, ascending)) - new SortedRDD(rangePartitionedRDD, ascending) + def sortByKey(ascending: Boolean = true, numSplits: Int = self.splits.size): RDD[(K,V)] = { + new ShuffledSortedRDD(self, ascending, numSplits) } } -class SortedRDD[K <% Ordered[K], V](prev: RDD[(K, V)], ascending: Boolean) - extends RDD[(K, V)](prev.context) { - - override def splits = prev.splits - override val partitioner = prev.partitioner - override val dependencies = List(new OneToOneDependency(prev)) - - override def compute(split: Split) = { - prev.iterator(split).toArray - .sortWith((x, y) => if (ascending) x._1 < y._1 else x._1 > y._1).iterator - } -} - class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) { override def splits = prev.splits override val dependencies = List(new OneToOneDependency(prev)) @@ -444,7 +449,7 @@ class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)] class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U]) extends RDD[(K, U)](prev.context) { - + override def splits = prev.splits override val dependencies = List(new OneToOneDependency(prev)) override val partitioner = prev.partitioner @@ -454,6 +459,6 @@ class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U] } } -object Manifests { +private[spark] object Manifests { val seqSeqManifest = classManifest[Seq[Seq[_]]] } diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala index d79007ab40..321f5264b8 100644 --- a/core/src/main/scala/spark/ParallelCollection.scala +++ b/core/src/main/scala/spark/ParallelCollection.scala @@ -3,7 +3,7 @@ package spark import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer -class ParallelCollectionSplit[T: ClassManifest]( +private[spark] class ParallelCollectionSplit[T: ClassManifest]( val rddId: Long, val slice: Int, values: Seq[T]) diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 643541429f..20c31714ae 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -41,9 +41,9 @@ class RangePartitioner[K <% Ordered[K]: ClassManifest, V]( Array() } else { val rddSize = rdd.count() - val maxSampleSize = partitions * 10.0 + val maxSampleSize = partitions * 20.0 val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0) - val rddSample = rdd.sample(true, frac, 1).map(_._1).collect().sortWith(_ < _) + val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sortWith(_ < _) if (rddSample.length == 0) { Array() } else { diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 3fe8e8a4bf..351c3d9d0b 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -61,6 +61,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial def compute(split: Split): Iterator[T] @transient val dependencies: List[Dependency[_]] + // Record user function generating this RDD + val origin = Utils.getSparkCallSite + // Optionally overridden by subclasses to specify how they are partitioned val partitioner: Option[Partitioner] = None @@ -68,6 +71,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial def preferredLocations(split: Split): Seq[String] = Nil def context = sc + + def elementClassManifest: ClassManifest[T] = classManifest[T] // Get a unique ID for this RDD val id = sc.newRddId() @@ -87,21 +92,21 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial } // Turn on the default caching level for this RDD - def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY_DESER) + def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) // Turn on the default caching level for this RDD def cache(): RDD[T] = persist() def getStorageLevel = storageLevel - def checkpoint(level: StorageLevel = StorageLevel.DISK_AND_MEMORY_DESER): RDD[T] = { + private[spark] def checkpoint(level: StorageLevel = StorageLevel.MEMORY_AND_DISK_2): RDD[T] = { if (!level.useDisk && level.replication < 2) { throw new Exception("Cannot checkpoint without using disk or replication (level requested was " + level + ")") } // This is a hack. Ideally this should re-use the code used by the CacheTracker // to generate the key. - def getSplitKey(split: Split) = "rdd:%d:%d".format(this.id, split.index) + def getSplitKey(split: Split) = "rdd_%d_%d".format(this.id, split.index) persist(level) sc.runJob(this, (iter: Iterator[T]) => {} ) @@ -131,7 +136,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) - def distinct(): RDD[T] = map(x => (x, "")).reduceByKey((x, y) => x).map(_._1) + def distinct(numSplits: Int = splits.size): RDD[T] = + map(x => (x, null)).reduceByKey((x, y) => x, numSplits).map(_._1) def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = new SampledRDD(this, withReplacement, fraction, seed) @@ -143,8 +149,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial var initialCount = count() var maxSelected = 0 - if (initialCount > Integer.MAX_VALUE) { - maxSelected = Integer.MAX_VALUE + if (initialCount > Integer.MAX_VALUE - 1) { + maxSelected = Integer.MAX_VALUE - 1 } else { maxSelected = initialCount.toInt } @@ -159,15 +165,14 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial total = num } - var samples = this.sample(withReplacement, fraction, seed).collect() + val rand = new Random(seed) + var samples = this.sample(withReplacement, fraction, rand.nextInt).collect() while (samples.length < total) { - samples = this.sample(withReplacement, fraction, seed).collect() + samples = this.sample(withReplacement, fraction, rand.nextInt).collect() } - val arr = samples.take(total) - - return arr + Utils.randomizeInPlace(samples, rand).take(total) } def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other)) @@ -195,6 +200,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U]): RDD[U] = new MapPartitionsRDD(this, sc.clean(f)) + def mapPartitionsWithSplit[U: ClassManifest](f: (Int, Iterator[T]) => Iterator[U]): RDD[U] = + new MapPartitionsWithSplitRDD(this, sc.clean(f)) + // Actions (launch a job to return a value to the user program) def foreach(f: T => Unit) { @@ -416,3 +424,18 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( override val dependencies = List(new OneToOneDependency(prev)) override def compute(split: Split) = f(prev.iterator(split)) } + +/** + * A variant of the MapPartitionsRDD that passes the split index into the + * closure. This can be used to generate or collect partition specific + * information such as the number of tuples in a partition. + */ +class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest]( + prev: RDD[T], + f: (Int, Iterator[T]) => Iterator[U]) + extends RDD[U](prev.context) { + + override def splits = prev.splits + override val dependencies = List(new OneToOneDependency(prev)) + override def compute(split: Split) = f(split.index, prev.iterator(split)) +} diff --git a/core/src/main/scala/spark/SampledRDD.scala b/core/src/main/scala/spark/SampledRDD.scala index 8ef40d8d9e..ac10aed477 100644 --- a/core/src/main/scala/spark/SampledRDD.scala +++ b/core/src/main/scala/spark/SampledRDD.scala @@ -1,7 +1,10 @@ package spark import java.util.Random +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand +private[spark] class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable { override val index: Int = prev.index } @@ -28,19 +31,21 @@ class SampledRDD[T: ClassManifest]( override def compute(splitIn: Split) = { val split = splitIn.asInstanceOf[SampledRDDSplit] - val rg = new Random(split.seed) - // Sampling with replacement (TODO: use reservoir sampling to make this more efficient?) if (withReplacement) { - val oldData = prev.iterator(split.prev).toArray - val sampleSize = (oldData.size * frac).ceil.toInt - val sampledData = { - // all of oldData's indices are candidates, even if sampleSize < oldData.size - for (i <- 1 to sampleSize) - yield oldData(rg.nextInt(oldData.size)) + // For large datasets, the expected number of occurrences of each element in a sample with + // replacement is Poisson(frac). We use that to get a count for each element. + val poisson = new Poisson(frac, new DRand(split.seed)) + prev.iterator(split.prev).flatMap { element => + val count = poisson.nextInt() + if (count == 0) { + Iterator.empty // Avoid object allocation when we return 0 items, which is quite often + } else { + Iterator.fill(count)(element) + } } - sampledData.iterator } else { // Sampling without replacement - prev.iterator(split.prev).filter(x => (rg.nextDouble <= frac)) + val rand = new Random(split.seed) + prev.iterator(split.prev).filter(x => (rand.nextDouble <= frac)) } } } diff --git a/core/src/main/scala/spark/Serializer.scala b/core/src/main/scala/spark/Serializer.scala index 61a70beaf1..c0e08289d8 100644 --- a/core/src/main/scala/spark/Serializer.scala +++ b/core/src/main/scala/spark/Serializer.scala @@ -12,14 +12,14 @@ import spark.util.ByteBufferInputStream * A serializer. Because some serialization libraries are not thread safe, this class is used to * create SerializerInstances that do the actual serialization. */ -trait Serializer { +private[spark] trait Serializer { def newInstance(): SerializerInstance } /** * An instance of the serializer, for use by one thread at a time. */ -trait SerializerInstance { +private[spark] trait SerializerInstance { def serialize[T](t: T): ByteBuffer def deserialize[T](bytes: ByteBuffer): T @@ -43,15 +43,15 @@ trait SerializerInstance { def deserializeMany(buffer: ByteBuffer): Iterator[Any] = { // Default implementation uses deserializeStream buffer.rewind() - deserializeStream(new ByteBufferInputStream(buffer)).toIterator + deserializeStream(new ByteBufferInputStream(buffer)).asIterator } } /** * A stream for writing serialized objects. */ -trait SerializationStream { - def writeObject[T](t: T): Unit +private[spark] trait SerializationStream { + def writeObject[T](t: T): SerializationStream def flush(): Unit def close(): Unit @@ -66,7 +66,7 @@ trait SerializationStream { /** * A stream for reading serialized objects. */ -trait DeserializationStream { +private[spark] trait DeserializationStream { def readObject[T](): T def close(): Unit @@ -74,7 +74,7 @@ trait DeserializationStream { * Read the elements of this stream through an iterator. This can only be called once, as * reading each element will consume data from the input source. */ - def toIterator: Iterator[Any] = new Iterator[Any] { + def asIterator: Iterator[Any] = new Iterator[Any] { var gotNext = false var finished = false var nextValue: Any = null diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 4f8d98f7d0..daa35fe7f2 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,6 +1,6 @@ package spark -abstract class ShuffleFetcher { +private[spark] abstract class ShuffleFetcher { // Fetch the shuffle outputs for a given ShuffleDependency, calling func exactly // once on each key-value pair obtained. def fetch[K, V](shuffleId: Int, reduceId: Int, func: (K, V) => Unit) diff --git a/core/src/main/scala/spark/ShuffleManager.scala b/core/src/main/scala/spark/ShuffleManager.scala deleted file mode 100644 index 24af7f3a08..0000000000 --- a/core/src/main/scala/spark/ShuffleManager.scala +++ /dev/null @@ -1,98 +0,0 @@ -package spark - -import java.io._ -import java.net.URL -import java.util.UUID -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import spark._ - -class ShuffleManager extends Logging { - private var nextShuffleId = new AtomicLong(0) - - private var shuffleDir: File = null - private var server: HttpServer = null - private var serverUri: String = null - - initialize() - - private def initialize() { - // TODO: localDir should be created by some mechanism common to Spark - // so that it can be shared among shuffle, broadcast, etc - val localDirRoot = System.getProperty("spark.local.dir", "/tmp") - var tries = 0 - var foundLocalDir = false - var localDir: File = null - var localDirUuid: UUID = null - while (!foundLocalDir && tries < 10) { - tries += 1 - try { - localDirUuid = UUID.randomUUID - localDir = new File(localDirRoot, "spark-local-" + localDirUuid) - if (!localDir.exists) { - localDir.mkdirs() - foundLocalDir = true - } - } catch { - case e: Exception => - logWarning("Attempt " + tries + " to create local dir failed", e) - } - } - if (!foundLocalDir) { - logError("Failed 10 attempts to create local dir in " + localDirRoot) - System.exit(1) - } - shuffleDir = new File(localDir, "shuffle") - shuffleDir.mkdirs() - logInfo("Shuffle dir: " + shuffleDir) - - // Add a shutdown hook to delete the local dir - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dir") { - override def run() { - Utils.deleteRecursively(localDir) - } - }) - - val extServerPort = System.getProperty( - "spark.localFileShuffle.external.server.port", "-1").toInt - if (extServerPort != -1) { - // We're using an external HTTP server; set URI relative to its root - var extServerPath = System.getProperty( - "spark.localFileShuffle.external.server.path", "") - if (extServerPath != "" && !extServerPath.endsWith("/")) { - extServerPath += "/" - } - serverUri = "http://%s:%d/%s/spark-local-%s".format( - Utils.localIpAddress, extServerPort, extServerPath, localDirUuid) - } else { - // Create our own server - server = new HttpServer(localDir) - server.start() - serverUri = server.uri - } - logInfo("Local URI: " + serverUri) - } - - def stop() { - if (server != null) { - server.stop() - } - } - - def getOutputFile(shuffleId: Long, inputId: Int, outputId: Int): File = { - val dir = new File(shuffleDir, shuffleId + "/" + inputId) - dir.mkdirs() - val file = new File(dir, "" + outputId) - return file - } - - def getServerUri(): String = { - serverUri - } - - def newShuffleId(): Long = { - nextShuffleId.getAndIncrement() - } -} diff --git a/core/src/main/scala/spark/ShuffledRDD.scala b/core/src/main/scala/spark/ShuffledRDD.scala index 3616d8e47e..1a9f4cfec3 100644 --- a/core/src/main/scala/spark/ShuffledRDD.scala +++ b/core/src/main/scala/spark/ShuffledRDD.scala @@ -1,29 +1,94 @@ package spark +import scala.collection.mutable.ArrayBuffer import java.util.{HashMap => JHashMap} -class ShuffledRDDSplit(val idx: Int) extends Split { + +private[spark] class ShuffledRDDSplit(val idx: Int) extends Split { override val index = idx override def hashCode(): Int = idx } -class ShuffledRDD[K, V, C]( + +/** + * The resulting RDD from a shuffle (e.g. repartitioning of data). + */ +abstract class ShuffledRDD[K, V, C]( @transient parent: RDD[(K, V)], aggregator: Aggregator[K, V, C], - part : Partitioner) + part: Partitioner) extends RDD[(K, C)](parent.context) { - //override val partitioner = Some(part) + override val partitioner = Some(part) - + @transient val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) override def splits = splits_ - + override def preferredLocations(split: Split) = Nil - + val dep = new ShuffleDependency(context.newShuffleId, parent, aggregator, part) override val dependencies = List(dep) +} + + +/** + * Repartition a key-value pair RDD. + */ +class RepartitionShuffledRDD[K, V]( + @transient parent: RDD[(K, V)], + part: Partitioner) + extends ShuffledRDD[K, V, V]( + parent, + Aggregator[K, V, V](null, null, null, false), + part) { + + override def compute(split: Split): Iterator[(K, V)] = { + val buf = new ArrayBuffer[(K, V)] + val fetcher = SparkEnv.get.shuffleFetcher + def addTupleToBuffer(k: K, v: V) = { buf += Tuple(k, v) } + fetcher.fetch[K, V](dep.shuffleId, split.index, addTupleToBuffer) + buf.iterator + } +} + + +/** + * A sort-based shuffle (that doesn't apply aggregation). It does so by first + * repartitioning the RDD by range, and then sort within each range. + */ +class ShuffledSortedRDD[K <% Ordered[K]: ClassManifest, V]( + @transient parent: RDD[(K, V)], + ascending: Boolean, + numSplits: Int) + extends RepartitionShuffledRDD[K, V]( + parent, + new RangePartitioner(numSplits, parent, ascending)) { + + override def compute(split: Split): Iterator[(K, V)] = { + // By separating this from RepartitionShuffledRDD, we avoided a + // buf.iterator.toArray call, thus avoiding building up the buffer twice. + val buf = new ArrayBuffer[(K, V)] + def addTupleToBuffer(k: K, v: V) { buf += ((k, v)) } + SparkEnv.get.shuffleFetcher.fetch[K, V](dep.shuffleId, split.index, addTupleToBuffer) + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + } +} + + +/** + * The resulting RDD from shuffle and running (hash-based) aggregation. + */ +class ShuffledAggregatedRDD[K, V, C]( + @transient parent: RDD[(K, V)], + aggregator: Aggregator[K, V, C], + part : Partitioner) + extends ShuffledRDD[K, V, C](parent, aggregator, part) { override def compute(split: Split): Iterator[(K, C)] = { val combiners = new JHashMap[K, C] diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index e5ad8b52dc..7c3e8640e9 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -22,7 +22,7 @@ import it.unimi.dsi.fastutil.ints.IntOpenHashSet * Based on the following JavaWorld article: * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html */ -object SizeEstimator extends Logging { +private[spark] object SizeEstimator extends Logging { // Sizes of primitive types private val BYTE_SIZE = 1 @@ -77,22 +77,18 @@ object SizeEstimator extends Logging { return System.getProperty("spark.test.useCompressedOops").toBoolean } try { - val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic"; - val server = ManagementFactory.getPlatformMBeanServer(); + val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic" + val server = ManagementFactory.getPlatformMBeanServer() val bean = ManagementFactory.newPlatformMXBeanProxy(server, - hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean]); + hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean]) return bean.getVMOption("UseCompressedOops").getValue.toBoolean } catch { - case e: IllegalArgumentException => { - logWarning("Exception while trying to check if compressed oops is enabled", e) - // Fall back to checking if maxMemory < 32GB - return Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) - } - - case e: SecurityException => { - logWarning("No permission to create MBeanServer", e) - // Fall back to checking if maxMemory < 32GB - return Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) + case e: Exception => { + // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB + val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) + val guessInWords = if (guess) "yes" else "not" + logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords) + return guess } } } @@ -146,6 +142,10 @@ object SizeEstimator extends Logging { val cls = obj.getClass if (cls.isArray) { visitArray(obj, cls, state) + } else if (obj.isInstanceOf[ClassLoader] || obj.isInstanceOf[Class[_]]) { + // Hadoop JobConfs created in the interpreter have a ClassLoader, which greatly confuses + // the size estimator since it references the whole REPL. Do nothing in this case. In + // general all ClassLoaders and Classes will be shared between objects anyway. } else { val classInfo = getClassInfo(cls) state.size += classInfo.shellSize diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala index ce9370c5d7..3dd0a4b1f9 100644 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ b/core/src/main/scala/spark/SoftReferenceCache.scala @@ -5,7 +5,7 @@ import com.google.common.collect.MapMaker /** * An implementation of Cache that uses soft references. */ -class SoftReferenceCache extends Cache { +private[spark] class SoftReferenceCache extends Cache { val map = new MapMaker().softValues().makeMap[Any, Any]() override def get(datasetId: Any, partition: Int): Any = diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 538e057926..83c1b49203 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -2,13 +2,15 @@ package spark import java.io._ import java.util.concurrent.atomic.AtomicInteger +import java.net.{URI, URLClassLoader} import akka.actor.Actor import akka.actor.Actor._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.generic.Growable -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileUtil, Path} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.SequenceFileInputFormat @@ -34,6 +36,8 @@ import org.apache.mesos.{Scheduler, MesosNativeLibrary} import spark.broadcast._ +import spark.deploy.LocalSparkCluster + import spark.partial.ApproximateEvaluator import spark.partial.PartialResult @@ -51,7 +55,7 @@ class SparkContext( val sparkHome: String, val jars: Seq[String]) extends Logging { - + def this(master: String, frameworkName: String) = this(master, frameworkName, null, Nil) // Ensure logging is initialized before we spawn any threads @@ -75,24 +79,33 @@ class SparkContext( isLocal) SparkEnv.set(env) + // Used to store a URL for each static file/jar together with the file's local timestamp + val addedFiles = HashMap[String, Long]() + val addedJars = HashMap[String, Long]() + + // Add each JAR given through the constructor + jars.foreach { addJar(_) } + // Create and start the scheduler private var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks - val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+),([0-9]+)\]""".r + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r + // Regular expression for simulating a Spark cluster of [N, cores, memory] locally + val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters val SPARK_REGEX = """(spark://.*)""".r master match { - case "local" => - new LocalScheduler(1, 0) + case "local" => + new LocalScheduler(1, 0, this) - case LOCAL_N_REGEX(threads) => - new LocalScheduler(threads.toInt, 0) + case LOCAL_N_REGEX(threads) => + new LocalScheduler(threads.toInt, 0, this) case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - new LocalScheduler(threads.toInt, maxFailures.toInt) + new LocalScheduler(threads.toInt, maxFailures.toInt, this) case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) @@ -100,6 +113,28 @@ class SparkContext( scheduler.initialize(backend) scheduler + case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => + // Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang. + val memoryPerSlaveInt = memoryPerSlave.toInt + val sparkMemEnv = System.getenv("SPARK_MEM") + val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512 + if (sparkMemEnvInt > memoryPerSlaveInt) { + throw new SparkException( + "Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format( + memoryPerSlaveInt, sparkMemEnvInt)) + } + + val scheduler = new ClusterScheduler(this) + val localCluster = new LocalSparkCluster( + numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) + val sparkUrl = localCluster.start() + val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, frameworkName) + scheduler.initialize(backend) + backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { + localCluster.stop() + } + scheduler + case _ => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(this) @@ -122,7 +157,7 @@ class SparkContext( def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = { new ParallelCollection[T](this, seq, numSlices) } - + def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = { parallelize(seq, numSlices) } @@ -163,14 +198,14 @@ class SparkContext( } /** - * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys, + * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys, * values and the InputFormat so that users don't need to pass them directly. */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int) (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]) : RDD[(K, V)] = { hadoopFile(path, - fm.erasure.asInstanceOf[Class[F]], + fm.erasure.asInstanceOf[Class[F]], km.erasure.asInstanceOf[Class[K]], vm.erasure.asInstanceOf[Class[V]], minSplits) @@ -191,7 +226,7 @@ class SparkContext( new Configuration) } - /** + /** * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. */ @@ -207,7 +242,7 @@ class SparkContext( new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf) } - /** + /** * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. */ @@ -233,14 +268,14 @@ class SparkContext( sequenceFile(path, keyClass, valueClass, defaultMinSplits) /** - * Version of sequenceFile() for types implicitly convertible to Writables through a + * Version of sequenceFile() for types implicitly convertible to Writables through a * WritableConverter. * * WritableConverters are provided in a somewhat strange way (by an implicit function) to support - * both subclasses of Writable and types for which we define a converter (e.g. Int to + * both subclasses of Writable and types for which we define a converter (e.g. Int to * IntWritable). The most natural thing would've been to have implicit objects for the * converters, but then we couldn't have an object for every subclass of Writable (you can't - * have a parameterized singleton object). We use functions instead to create a new converter + * have a parameterized singleton object). We use functions instead to create a new converter * for the appropriate type. In addition, we pass the converter a ClassManifest of its type to * allow it to figure out the Writable class to use in the subclass case. */ @@ -265,7 +300,7 @@ class SparkContext( * that there's very little effort required to save arbitrary objects. */ def objectFile[T: ClassManifest]( - path: String, + path: String, minSplits: Int = defaultMinSplits ): RDD[T] = { sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minSplits) @@ -292,10 +327,57 @@ class SparkContext( def accumulable[T,R](initialValue: T)(implicit param: AccumulableParam[T,R]) = new Accumulable(initialValue, param) + /** + * Create an accumulator from a "mutable collection" type. + * + * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by + * standard mutable collections. So you can use this with mutable Map, Set, etc. + */ + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T](initialValue: R) = { + val param = new GrowableAccumulableParam[R,T] + new Accumulable(initialValue, param) + } // Keep around a weak hash map of values to Cached versions? def broadcast[T](value: T) = SparkEnv.get.broadcastManager.newBroadcast[T] (value, isLocal) + // Adds a file dependency to all Tasks executed in the future. + def addFile(path: String) { + val uri = new URI(path) + val key = uri.getScheme match { + case null | "file" => env.httpFileServer.addFile(new File(uri.getPath)) + case _ => path + } + addedFiles(key) = System.currentTimeMillis + + // Fetch the file locally in case the task is executed locally + val filename = new File(path.split("/").last) + Utils.fetchFile(path, new File(".")) + + logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) + } + + def clearFiles() { + addedFiles.keySet.map(_.split("/").last).foreach { k => new File(k).delete() } + addedFiles.clear() + } + + // Adds a jar dependency to all Tasks executed in the future. + def addJar(path: String) { + val uri = new URI(path) + val key = uri.getScheme match { + case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case _ => path + } + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + } + + def clearJars() { + addedJars.keySet.map(_.split("/").last).foreach { k => new File(k).delete() } + addedJars.clear() + } + // Stop the SparkContext def stop() { dagScheduler.stop() @@ -303,6 +385,9 @@ class SparkContext( taskScheduler = null // TODO: Cache.stop()? env.stop() + // Clean up locally linked files + clearFiles() + clearJars() SparkEnv.set(null) ShuffleMapTask.clearCache() logInfo("Successfully stopped SparkContext") @@ -326,7 +411,7 @@ class SparkContext( /** * Run a function on a given set of partitions in an RDD and return the results. This is the main * entry point to the scheduler, by which all actions get launched. The allowLocal flag specifies - * whether the scheduler can run the computation on the master rather than shipping it out to the + * whether the scheduler can run the computation on the master rather than shipping it out to the * cluster, for short actions like first(). */ def runJob[T, U: ClassManifest]( @@ -335,22 +420,23 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean ): Array[U] = { - logInfo("Starting job...") + val callSite = Utils.getSparkCallSite + logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, allowLocal) - logInfo("Job finished in " + (System.nanoTime - start) / 1e9 + " s") + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal) + logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") result } def runJob[T, U: ClassManifest]( rdd: RDD[T], - func: Iterator[T] => U, + func: Iterator[T] => U, partitions: Seq[Int], allowLocal: Boolean ): Array[U] = { runJob(rdd, (context: TaskContext, iter: Iterator[T]) => func(iter), partitions, allowLocal) } - + /** * Run a job on all partitions in an RDD and return the results in an array. */ @@ -371,10 +457,11 @@ class SparkContext( evaluator: ApproximateEvaluator[U, R], timeout: Long ): PartialResult[R] = { - logInfo("Starting job...") + val callSite = Utils.getSparkCallSite + logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runApproximateJob(rdd, func, evaluator, timeout) - logInfo("Job finished in " + (System.nanoTime - start) / 1e9 + " s") + val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout) + logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") result } @@ -396,7 +483,7 @@ class SparkContext( private[spark] def newShuffleId(): Int = { nextShuffleId.getAndIncrement() } - + private var nextRddId = new AtomicInteger(0) // Register a new RDD, returning its RDD ID @@ -424,7 +511,7 @@ object SparkContext { implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = new PairRDDFunctions(rdd) - + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) @@ -445,7 +532,7 @@ object SparkContext { implicit def longToLongWritable(l: Long) = new LongWritable(l) implicit def floatToFloatWritable(f: Float) = new FloatWritable(f) - + implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d) implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b) @@ -456,7 +543,7 @@ object SparkContext { private implicit def arrayToArrayWritable[T <% Writable: ClassManifest](arr: Traversable[T]): ArrayWritable = { def anyToWritable[U <% Writable](u: U): Writable = u - + new ArrayWritable(classManifest[T].erasure.asInstanceOf[Class[Writable]], arr.map(x => anyToWritable(x)).toArray) } @@ -500,7 +587,7 @@ object SparkContext { Nil } } - + // Find the JAR that contains the class of a particular object def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) } @@ -513,7 +600,7 @@ object SparkContext { * that doesn't know the type of T when it is created. This sounds strange but is necessary to * support converting subclasses of Writable to themselves (writableWritableConverter). */ -class WritableConverter[T]( +private[spark] class WritableConverter[T]( val writableClass: ClassManifest[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index add8fcec51..f2a52ab356 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,6 +1,8 @@ package spark import akka.actor.ActorSystem +import akka.actor.ActorSystemImpl +import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -8,35 +10,45 @@ import spark.storage.BlockManagerMaster import spark.network.ConnectionManager import spark.util.AkkaUtils +/** + * Holds all the runtime environment objects for a running Spark instance (either master or worker), + * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently + * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these + * objects needs to have the right SparkEnv set. You can get the current environment with + * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + */ class SparkEnv ( val actorSystem: ActorSystem, - val cache: Cache, val serializer: Serializer, val closureSerializer: Serializer, val cacheTracker: CacheTracker, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, - val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, val blockManager: BlockManager, - val connectionManager: ConnectionManager + val connectionManager: ConnectionManager, + val httpFileServer: HttpFileServer ) { /** No-parameter constructor for unit tests. */ def this() = { - this(null, null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null) + this(null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null) } def stop() { + httpFileServer.stop() mapOutputTracker.stop() cacheTracker.stop() shuffleFetcher.stop() - shuffleManager.stop() broadcastManager.stop() blockManager.stop() blockManager.master.stop() actorSystem.shutdown() + // Akka's awaitTermination doesn't actually wait until the port is unbound, so sleep a bit + Thread.sleep(100) actorSystem.awaitTermination() + // Akka's awaitTermination doesn't actually wait until the port is unbound, so sleep a bit + Thread.sleep(100) } } @@ -66,66 +78,49 @@ object SparkEnv { System.setProperty("spark.master.port", boundPort.toString) } - val serializerClass = System.getProperty("spark.serializer", "spark.KryoSerializer") - val serializer = Class.forName(serializerClass).newInstance().asInstanceOf[Serializer] + val classLoader = Thread.currentThread.getContextClassLoader + + // Create an instance of the class named by the given Java system property, or by + // defaultClassName if the property is not set, and return it as a T + def instantiateClass[T](propertyName: String, defaultClassName: String): T = { + val name = System.getProperty(propertyName, defaultClassName) + Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] + } + + val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") val blockManagerMaster = new BlockManagerMaster(actorSystem, isMaster, isLocal) - val blockManager = new BlockManager(blockManagerMaster, serializer) - val connectionManager = blockManager.connectionManager - - val shuffleManager = new ShuffleManager() + val connectionManager = blockManager.connectionManager val broadcastManager = new BroadcastManager(isMaster) - val closureSerializerClass = - System.getProperty("spark.closure.serializer", "spark.JavaSerializer") - val closureSerializer = - Class.forName(closureSerializerClass).newInstance().asInstanceOf[Serializer] - val cacheClass = System.getProperty("spark.cache.class", "spark.BoundedMemoryCache") - val cache = Class.forName(cacheClass).newInstance().asInstanceOf[Cache] + val closureSerializer = instantiateClass[Serializer]( + "spark.closure.serializer", "spark.JavaSerializer") val cacheTracker = new CacheTracker(actorSystem, isMaster, blockManager) blockManager.cacheTracker = cacheTracker val mapOutputTracker = new MapOutputTracker(actorSystem, isMaster) - val shuffleFetcherClass = - System.getProperty("spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") - val shuffleFetcher = - Class.forName(shuffleFetcherClass).newInstance().asInstanceOf[ShuffleFetcher] - - /* - if (System.getProperty("spark.stream.distributed", "false") == "true") { - val blockManagerClass = classOf[spark.storage.BlockManager].asInstanceOf[Class[_]] - if (isLocal || !isMaster) { - (new Thread() { - override def run() { - println("Wait started") - Thread.sleep(60000) - println("Wait ended") - val receiverClass = Class.forName("spark.stream.TestStreamReceiver4") - val constructor = receiverClass.getConstructor(blockManagerClass) - val receiver = constructor.newInstance(blockManager) - receiver.asInstanceOf[Thread].start() - } - }).start() - } - } - */ + val shuffleFetcher = instantiateClass[ShuffleFetcher]( + "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") + + val httpFileServer = new HttpFileServer() + httpFileServer.initialize() + System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) new SparkEnv( actorSystem, - cache, serializer, closureSerializer, cacheTracker, mapOutputTracker, shuffleFetcher, - shuffleManager, broadcastManager, blockManager, - connectionManager) + connectionManager, + httpFileServer) } } diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 6e4eb25ed4..420c54bc9a 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -7,10 +7,16 @@ import spark.storage.BlockManagerId * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -sealed trait TaskEndReason +private[spark] sealed trait TaskEndReason -case object Success extends TaskEndReason +private[spark] case object Success extends TaskEndReason + +private[spark] case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it + +private[spark] case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -case class ExceptionFailure(exception: Throwable) extends TaskEndReason -case class OtherFailure(message: String) extends TaskEndReason + +private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason + +private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 9566b52432..78eb33a628 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -2,7 +2,7 @@ package spark import org.apache.mesos.Protos.{TaskState => MesosTaskState} -object TaskState +private[spark] object TaskState extends Enumeration("LAUNCHING", "RUNNING", "FINISHED", "FAILED", "KILLED", "LOST") { val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value diff --git a/core/src/main/scala/spark/UnionRDD.scala b/core/src/main/scala/spark/UnionRDD.scala index 0e8164d6ab..3e795ea2a2 100644 --- a/core/src/main/scala/spark/UnionRDD.scala +++ b/core/src/main/scala/spark/UnionRDD.scala @@ -2,7 +2,7 @@ package spark import scala.collection.mutable.ArrayBuffer -class UnionSplit[T: ClassManifest]( +private[spark] class UnionSplit[T: ClassManifest]( idx: Int, rdd: RDD[T], split: Split) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 5eda1011f9..a480fe046d 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,18 +1,18 @@ package spark import java.io._ -import java.net.InetAddress +import java.net.{InetAddress, URL, URI} +import java.util.{Locale, Random, UUID} import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} - +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import scala.collection.mutable.ArrayBuffer -import scala.util.Random -import java.util.{Locale, UUID} import scala.io.Source /** * Various utility methods used by Spark. */ -object Utils { +private object Utils extends Logging { /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -116,22 +116,75 @@ object Utils { copyStream(in, out, true) } + /** Download a file from a given URL to the local filesystem */ + def downloadFile(url: URL, localPath: String) { + val in = url.openStream() + val out = new FileOutputStream(localPath) + Utils.copyStream(in, out, true) + } + + /** + * Download a file requested by the executor. Supports fetching the file in a variety of ways, + * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + */ + def fetchFile(url: String, targetDir: File) { + val filename = url.split("/").last + val targetFile = new File(targetDir, filename) + val uri = new URI(url) + uri.getScheme match { + case "http" | "https" | "ftp" => + logInfo("Fetching " + url + " to " + targetFile) + val in = new URL(url).openStream() + val out = new FileOutputStream(targetFile) + Utils.copyStream(in, out, true) + case "file" | null => + // Remove the file if it already exists + targetFile.delete() + // Symlink the file locally + logInfo("Symlinking " + url + " to " + targetFile) + FileUtil.symLink(url, targetFile.toString) + case _ => + // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others + val uri = new URI(url) + val conf = new Configuration() + val fs = FileSystem.get(uri, conf) + val in = fs.open(new Path(uri)) + val out = new FileOutputStream(targetFile) + Utils.copyStream(in, out, true) + } + // Decompress the file if it's a .tar or .tar.gz + if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { + logInfo("Untarring " + filename) + Utils.execute(Seq("tar", "-xzf", filename), targetDir) + } else if (filename.endsWith(".tar")) { + logInfo("Untarring " + filename) + Utils.execute(Seq("tar", "-xf", filename), targetDir) + } + // Make the file executable - That's necessary for scripts + FileUtil.chmod(filename, "a+x") + } + /** * Shuffle the elements of a collection into a random order, returning the * result in a new collection. Unlike scala.util.Random.shuffle, this method * uses a local random number generator, avoiding inter-thread contention. */ - def randomize[T](seq: TraversableOnce[T]): Seq[T] = { - val buf = new ArrayBuffer[T]() - buf ++= seq - val rand = new Random() - for (i <- (buf.size - 1) to 1 by -1) { + def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = { + randomizeInPlace(seq.toArray) + } + + /** + * Shuffle the elements of an array into a random order, modifying the + * original array. Returns the original array. + */ + def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = { + for (i <- (arr.length - 1) to 1 by -1) { val j = rand.nextInt(i) - val tmp = buf(j) - buf(j) = buf(i) - buf(i) = tmp + val tmp = arr(j) + arr(j) = arr(i) + arr(i) = tmp } - buf + arr } /** @@ -294,4 +347,43 @@ object Utils { def execute(command: Seq[String]) { execute(command, new File(".")) } + + + /** + * When called inside a class in the spark package, returns the name of the user code class + * (outside the spark package) that called into Spark, as well as which Spark method they called. + * This is used, for example, to tell users where in their code each RDD got created. + */ + def getSparkCallSite: String = { + val trace = Thread.currentThread.getStackTrace().filter( el => + (!el.getMethodName.contains("getStackTrace"))) + + // Keep crawling up the stack trace until we find the first function not inside of the spark + // package. We track the last (shallowest) contiguous Spark method. This might be an RDD + // transformation, a SparkContext function (such as parallelize), or anything else that leads + // to instantiation of an RDD. We also track the first (deepest) user method, file, and line. + var lastSparkMethod = "<unknown>" + var firstUserFile = "<unknown>" + var firstUserLine = 0 + var finished = false + + for (el <- trace) { + if (!finished) { + if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) { + lastSparkMethod = if (el.getMethodName == "<init>") { + // Spark method is a constructor; get its class name + el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) + } else { + el.getMethodName + } + } + else { + firstUserLine = el.getLineNumber + firstUserFile = el.getFileName + finished = true + } + } + } + "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) + } } diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 7c0b17c45e..9731bb4eac 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -33,6 +33,8 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def distinct(): JavaDoubleRDD = fromRDD(srdd.distinct()) + def distinct(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numSplits)) + def filter(f: JFunction[Double, java.lang.Boolean]): JavaDoubleRDD = fromRDD(srdd.filter(x => f(x).booleanValue())) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c28a13b061..84ec386ce4 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -40,6 +40,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def distinct(): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct()) + def distinct(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numSplits)) + def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 541aa1e60b..b3e1977bcb 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -19,6 +19,8 @@ JavaRDDLike[T, JavaRDD[T]] { def distinct(): JavaRDD[T] = wrapRDD(rdd.distinct()) + def distinct(numSplits: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numSplits)) + def filter(f: JFunction[T, java.lang.Boolean]): JavaRDD[T] = wrapRDD(rdd.filter((x => f(x).booleanValue()))) diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala index d08e1e9fbf..923f5cdf4f 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala @@ -7,7 +7,7 @@ import scala.runtime.AbstractFunction1 * apply() method as call() and declare that it can throw Exception (since AbstractFunction1.apply * isn't marked to allow that). */ -abstract class WrappedFunction1[T, R] extends AbstractFunction1[T, R] { +private[spark] abstract class WrappedFunction1[T, R] extends AbstractFunction1[T, R] { @throws(classOf[Exception]) def call(t: T): R diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala index c9d67d9771..2c6e9b1571 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala @@ -7,7 +7,7 @@ import scala.runtime.AbstractFunction2 * apply() method as call() and declare that it can throw Exception (since AbstractFunction2.apply * isn't marked to allow that). */ -abstract class WrappedFunction2[T1, T2, R] extends AbstractFunction2[T1, T2, R] { +private[spark] abstract class WrappedFunction2[T1, T2, R] extends AbstractFunction2[T1, T2, R] { @throws(classOf[Exception]) def call(t1: T1, t2: T2): R diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index 0715722dc1..ef27bbb502 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -11,14 +11,17 @@ import scala.math import spark._ import spark.storage.StorageLevel -class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean) -extends Broadcast[T] with Logging with Serializable { +private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) + extends Broadcast[T](id) + with Logging + with Serializable { def value = value_ + def blockId: String = "broadcast_" + id + MultiTracker.synchronized { - SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } @transient var arrayOfBlocks: Array[BroadcastBlock] = null @@ -45,7 +48,7 @@ extends Broadcast[T] with Logging with Serializable { // Used only in Workers @transient var ttGuide: TalkToGuide = null - @transient var hostAddress = Utils.localIpAddress + @transient var hostAddress = Utils.localIpAddress() @transient var listenPort = -1 @transient var guidePort = -1 @@ -53,7 +56,7 @@ extends Broadcast[T] with Logging with Serializable { // Must call this after all the variables have been created/initialized if (!isLocal) { - sendBroadcast + sendBroadcast() } def sendBroadcast() { @@ -106,20 +109,22 @@ extends Broadcast[T] with Logging with Serializable { listOfSources += masterSource // Register with the Tracker - MultiTracker.registerBroadcast(uuid, + MultiTracker.registerBroadcast(id, SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes)) } private def readObject(in: ObjectInputStream) { in.defaultReadObject() MultiTracker.synchronized { - SparkEnv.get.blockManager.getSingle(uuid.toString) match { - case Some(x) => x.asInstanceOf[T] - case None => { - logInfo("Started reading broadcast variable " + uuid) + SparkEnv.get.blockManager.getSingle(blockId) match { + case Some(x) => + value_ = x.asInstanceOf[T] + + case None => + logInfo("Started reading broadcast variable " + id) // Initializing everything because Master will only send null/0 values // Only the 1st worker in a node can be here. Others will get from cache - initializeWorkerVariables + initializeWorkerVariables() logInfo("Local host address: " + hostAddress) @@ -131,18 +136,17 @@ extends Broadcast[T] with Logging with Serializable { val start = System.nanoTime - val receptionSucceeded = receiveBroadcast(uuid) + val receptionSucceeded = receiveBroadcast(id) if (receptionSucceeded) { value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } else { - logError("Reading Broadcasted variable " + uuid + " failed") + logError("Reading broadcast variable " + id + " failed") } val time = (System.nanoTime - start) / 1e9 - logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") - } + logInfo("Reading broadcast variable " + id + " took " + time + " s") } } } @@ -254,8 +258,8 @@ extends Broadcast[T] with Logging with Serializable { } } - def receiveBroadcast(variableUUID: UUID): Boolean = { - val gInfo = MultiTracker.getGuideInfo(variableUUID) + def receiveBroadcast(variableID: Long): Boolean = { + val gInfo = MultiTracker.getGuideInfo(variableID) if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) { return false @@ -764,7 +768,7 @@ extends Broadcast[T] with Logging with Serializable { logInfo("Sending stopBroadcast notifications...") sendStopBroadcastNotifications - MultiTracker.unregisterBroadcast(uuid) + MultiTracker.unregisterBroadcast(id) } finally { if (serverSocket != null) { logInfo("GuideMultipleRequests now stopping...") @@ -1025,9 +1029,12 @@ extends Broadcast[T] with Logging with Serializable { } } -class BitTorrentBroadcastFactory +private[spark] class BitTorrentBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) = MultiTracker.initialize(isMaster) - def newBroadcast[T](value_ : T, isLocal: Boolean) = new BitTorrentBroadcast[T](value_, isLocal) - def stop() = MultiTracker.stop + def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new BitTorrentBroadcast[T](value_, isLocal, id) + + def stop() { MultiTracker.stop() } } diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index d68e56a114..6055bfd045 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -1,25 +1,20 @@ package spark.broadcast import java.io._ -import java.net._ -import java.util.{BitSet, UUID} -import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} - -import scala.collection.mutable.Map +import java.util.concurrent.atomic.AtomicLong import spark._ -trait Broadcast[T] extends Serializable { - val uuid = UUID.randomUUID - +abstract class Broadcast[T](id: Long) extends Serializable { def value: T // We cannot have an abstract readObject here due to some weird issues with // readObject having to be 'private' in sub-classes. - override def toString = "spark.Broadcast(" + uuid + ")" + override def toString = "spark.Broadcast(" + id + ")" } +private[spark] class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializable { private var initialized = false @@ -49,14 +44,10 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl broadcastFactory.stop() } - private def getBroadcastFactory: BroadcastFactory = { - if (broadcastFactory == null) { - throw new SparkException ("Broadcast.getBroadcastFactory called before initialize") - } - broadcastFactory - } + private val nextBroadcastId = new AtomicLong(0) - def newBroadcast[T](value_ : T, isLocal: Boolean) = broadcastFactory.newBroadcast[T](value_, isLocal) + def newBroadcast[T](value_ : T, isLocal: Boolean) = + broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) def isMaster = isMaster_ } diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala index e341d556bf..ab6d302827 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala @@ -6,8 +6,8 @@ package spark.broadcast * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ -trait BroadcastFactory { +private[spark] trait BroadcastFactory { def initialize(isMaster: Boolean): Unit - def newBroadcast[T](value_ : T, isLocal: Boolean): Broadcast[T] + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index 03986ea756..7eb4ddb74f 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -12,44 +12,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import spark._ import spark.storage.StorageLevel -class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean) -extends Broadcast[T] with Logging with Serializable { +private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +extends Broadcast[T](id) with Logging with Serializable { def value = value_ + def blockId: String = "broadcast_" + id + HttpBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } if (!isLocal) { - HttpBroadcast.write(uuid, value_) + HttpBroadcast.write(id, value_) } // Called by JVM when deserializing an object private def readObject(in: ObjectInputStream) { in.defaultReadObject() HttpBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(uuid.toString) match { + SparkEnv.get.blockManager.getSingle(blockId) match { case Some(x) => value_ = x.asInstanceOf[T] case None => { - logInfo("Started reading broadcast variable " + uuid) + logInfo("Started reading broadcast variable " + id) val start = System.nanoTime - value_ = HttpBroadcast.read[T](uuid) - SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + value_ = HttpBroadcast.read[T](id) + SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) val time = (System.nanoTime - start) / 1e9 - logInfo("Reading broadcast variable " + uuid + " took " + time + " s") + logInfo("Reading broadcast variable " + id + " took " + time + " s") } } } } } -class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) = HttpBroadcast.initialize(isMaster) - def newBroadcast[T](value_ : T, isLocal: Boolean) = new HttpBroadcast[T](value_, isLocal) - def stop() = HttpBroadcast.stop() +private[spark] class HttpBroadcastFactory extends BroadcastFactory { + def initialize(isMaster: Boolean) { HttpBroadcast.initialize(isMaster) } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new HttpBroadcast[T](value_, isLocal, id) + + def stop() { HttpBroadcast.stop() } } private object HttpBroadcast extends Logging { @@ -65,7 +68,7 @@ private object HttpBroadcast extends Logging { synchronized { if (!initialized) { bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - compress = System.getProperty("spark.compress", "false").toBoolean + compress = System.getProperty("spark.broadcast.compress", "true").toBoolean if (isMaster) { createServer() } @@ -76,9 +79,12 @@ private object HttpBroadcast extends Logging { } def stop() { - if (server != null) { - server.stop() - server = null + synchronized { + if (server != null) { + server.stop() + server = null + } + initialized = false } } @@ -91,8 +97,8 @@ private object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } - def write(uuid: UUID, value: Any) { - val file = new File(broadcastDir, "broadcast-" + uuid) + def write(id: Long, value: Any) { + val file = new File(broadcastDir, "broadcast-" + id) val out: OutputStream = if (compress) { new LZFOutputStream(new FileOutputStream(file)) // Does its own buffering } else { @@ -104,8 +110,8 @@ private object HttpBroadcast extends Logging { serOut.close() } - def read[T](uuid: UUID): T = { - val url = serverUri + "/broadcast-" + uuid + def read[T](id: Long): T = { + val url = serverUri + "/broadcast-" + id var in = if (compress) { new LZFInputStream(new URL(url).openStream()) // Does its own buffering } else { diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala index a74677a777..5e76dedb94 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala @@ -2,8 +2,7 @@ package spark.broadcast import java.io._ import java.net._ -import java.util.{UUID, Random} -import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} +import java.util.Random import scala.collection.mutable.Map @@ -18,7 +17,7 @@ extends Logging { val FIND_BROADCAST_TRACKER = 2 // Map to keep track of guides of ongoing broadcasts - var valueToGuideMap = Map[UUID, SourceInfo]() + var valueToGuideMap = Map[Long, SourceInfo]() // Random number generator var ranGen = new Random @@ -154,44 +153,44 @@ extends Logging { val messageType = ois.readObject.asInstanceOf[Int] if (messageType == REGISTER_BROADCAST_TRACKER) { - // Receive UUID - val uuid = ois.readObject.asInstanceOf[UUID] + // Receive Long + val id = ois.readObject.asInstanceOf[Long] // Receive hostAddress and listenPort val gInfo = ois.readObject.asInstanceOf[SourceInfo] // Add to the map valueToGuideMap.synchronized { - valueToGuideMap += (uuid -> gInfo) + valueToGuideMap += (id -> gInfo) } - logInfo ("New broadcast " + uuid + " registered with TrackMultipleValues. Ongoing ones: " + valueToGuideMap) + logInfo ("New broadcast " + id + " registered with TrackMultipleValues. Ongoing ones: " + valueToGuideMap) // Send dummy ACK oos.writeObject(-1) oos.flush() } else if (messageType == UNREGISTER_BROADCAST_TRACKER) { - // Receive UUID - val uuid = ois.readObject.asInstanceOf[UUID] + // Receive Long + val id = ois.readObject.asInstanceOf[Long] // Remove from the map valueToGuideMap.synchronized { - valueToGuideMap(uuid) = SourceInfo("", SourceInfo.TxOverGoToDefault) + valueToGuideMap(id) = SourceInfo("", SourceInfo.TxOverGoToDefault) } - logInfo ("Broadcast " + uuid + " unregistered from TrackMultipleValues. Ongoing ones: " + valueToGuideMap) + logInfo ("Broadcast " + id + " unregistered from TrackMultipleValues. Ongoing ones: " + valueToGuideMap) // Send dummy ACK oos.writeObject(-1) oos.flush() } else if (messageType == FIND_BROADCAST_TRACKER) { - // Receive UUID - val uuid = ois.readObject.asInstanceOf[UUID] + // Receive Long + val id = ois.readObject.asInstanceOf[Long] var gInfo = - if (valueToGuideMap.contains(uuid)) valueToGuideMap(uuid) + if (valueToGuideMap.contains(id)) valueToGuideMap(id) else SourceInfo("", SourceInfo.TxNotStartedRetry) - logDebug("Got new request: " + clientSocket + " for " + uuid + " : " + gInfo.listenPort) + logDebug("Got new request: " + clientSocket + " for " + id + " : " + gInfo.listenPort) // Send reply back oos.writeObject(gInfo) @@ -224,7 +223,7 @@ extends Logging { } } - def getGuideInfo(variableUUID: UUID): SourceInfo = { + def getGuideInfo(variableLong: Long): SourceInfo = { var clientSocketToTracker: Socket = null var oosTracker: ObjectOutputStream = null var oisTracker: ObjectInputStream = null @@ -247,8 +246,8 @@ extends Logging { oosTracker.writeObject(MultiTracker.FIND_BROADCAST_TRACKER) oosTracker.flush() - // Send UUID and receive GuideInfo - oosTracker.writeObject(variableUUID) + // Send Long and receive GuideInfo + oosTracker.writeObject(variableLong) oosTracker.flush() gInfo = oisTracker.readObject.asInstanceOf[SourceInfo] } catch { @@ -276,7 +275,7 @@ extends Logging { return gInfo } - def registerBroadcast(uuid: UUID, gInfo: SourceInfo) { + def registerBroadcast(id: Long, gInfo: SourceInfo) { val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort) val oosST = new ObjectOutputStream(socket.getOutputStream) oosST.flush() @@ -286,8 +285,8 @@ extends Logging { oosST.writeObject(REGISTER_BROADCAST_TRACKER) oosST.flush() - // Send UUID of this broadcast - oosST.writeObject(uuid) + // Send Long of this broadcast + oosST.writeObject(id) oosST.flush() // Send this tracker's information @@ -303,7 +302,7 @@ extends Logging { socket.close() } - def unregisterBroadcast(uuid: UUID) { + def unregisterBroadcast(id: Long) { val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort) val oosST = new ObjectOutputStream(socket.getOutputStream) oosST.flush() @@ -313,8 +312,8 @@ extends Logging { oosST.writeObject(UNREGISTER_BROADCAST_TRACKER) oosST.flush() - // Send UUID of this broadcast - oosST.writeObject(uuid) + // Send Long of this broadcast + oosST.writeObject(id) oosST.flush() // Receive ACK and throw it away @@ -383,10 +382,10 @@ extends Logging { } } -case class BroadcastBlock(blockID: Int, byteArray: Array[Byte]) +private[spark] case class BroadcastBlock(blockID: Int, byteArray: Array[Byte]) extends Serializable -case class VariableInfo(@transient arrayOfBlocks : Array[BroadcastBlock], +private[spark] case class VariableInfo(@transient arrayOfBlocks : Array[BroadcastBlock], totalBlocks: Int, totalBytes: Int) extends Serializable { diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala index bf4027794d..c79bb93c38 100644 --- a/core/src/main/scala/spark/broadcast/SourceInfo.scala +++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala @@ -7,7 +7,7 @@ import spark._ /** * Used to keep and pass around information of peers involved in a broadcast */ -case class SourceInfo (hostAddress: String, +private[spark] case class SourceInfo (hostAddress: String, listenPort: Int, totalBlocks: Int = SourceInfo.UnusedParam, totalBytes: Int = SourceInfo.UnusedParam) @@ -26,7 +26,7 @@ extends Comparable[SourceInfo] with Logging { /** * Helper Object of SourceInfo for its constants */ -object SourceInfo { +private[spark] object SourceInfo { // Broadcast has not started yet! Should never happen. val TxNotStartedRetry = -1 // Broadcast has already finished. Try default mechanism. diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index 97e7d0120e..fa676e9064 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -10,14 +10,15 @@ import scala.math import spark._ import spark.storage.StorageLevel -class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean) -extends Broadcast[T] with Logging with Serializable { +private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +extends Broadcast[T](id) with Logging with Serializable { def value = value_ + def blockId = "broadcast_" + id + MultiTracker.synchronized { - SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } @transient var arrayOfBlocks: Array[BroadcastBlock] = null @@ -35,7 +36,7 @@ extends Broadcast[T] with Logging with Serializable { @transient var serveMR: ServeMultipleRequests = null @transient var guideMR: GuideMultipleRequests = null - @transient var hostAddress = Utils.localIpAddress + @transient var hostAddress = Utils.localIpAddress() @transient var listenPort = -1 @transient var guidePort = -1 @@ -43,7 +44,7 @@ extends Broadcast[T] with Logging with Serializable { // Must call this after all the variables have been created/initialized if (!isLocal) { - sendBroadcast + sendBroadcast() } def sendBroadcast() { @@ -84,20 +85,22 @@ extends Broadcast[T] with Logging with Serializable { listOfSources += masterSource // Register with the Tracker - MultiTracker.registerBroadcast(uuid, + MultiTracker.registerBroadcast(id, SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes)) } private def readObject(in: ObjectInputStream) { in.defaultReadObject() MultiTracker.synchronized { - SparkEnv.get.blockManager.getSingle(uuid.toString) match { - case Some(x) => x.asInstanceOf[T] - case None => { - logInfo("Started reading broadcast variable " + uuid) + SparkEnv.get.blockManager.getSingle(blockId) match { + case Some(x) => + value_ = x.asInstanceOf[T] + + case None => + logInfo("Started reading broadcast variable " + id) // Initializing everything because Master will only send null/0 values // Only the 1st worker in a node can be here. Others will get from cache - initializeWorkerVariables + initializeWorkerVariables() logInfo("Local host address: " + hostAddress) @@ -108,18 +111,17 @@ extends Broadcast[T] with Logging with Serializable { val start = System.nanoTime - val receptionSucceeded = receiveBroadcast(uuid) + val receptionSucceeded = receiveBroadcast(id) if (receptionSucceeded) { value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) SparkEnv.get.blockManager.putSingle( - uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false) + blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } else { - logError("Reading Broadcasted variable " + uuid + " failed") + logError("Reading broadcast variable " + id + " failed") } val time = (System.nanoTime - start) / 1e9 - logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") - } + logInfo("Reading broadcast variable " + id + " took " + time + " s") } } } @@ -136,14 +138,14 @@ extends Broadcast[T] with Logging with Serializable { serveMR = null - hostAddress = Utils.localIpAddress + hostAddress = Utils.localIpAddress() listenPort = -1 stopBroadcast = false } - def receiveBroadcast(variableUUID: UUID): Boolean = { - val gInfo = MultiTracker.getGuideInfo(variableUUID) + def receiveBroadcast(variableID: Long): Boolean = { + val gInfo = MultiTracker.getGuideInfo(variableID) if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) { return false @@ -318,7 +320,7 @@ extends Broadcast[T] with Logging with Serializable { logInfo("Sending stopBroadcast notifications...") sendStopBroadcastNotifications - MultiTracker.unregisterBroadcast(uuid) + MultiTracker.unregisterBroadcast(id) } finally { if (serverSocket != null) { logInfo("GuideMultipleRequests now stopping...") @@ -572,9 +574,12 @@ extends Broadcast[T] with Logging with Serializable { } } -class TreeBroadcastFactory +private[spark] class TreeBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) = MultiTracker.initialize(isMaster) - def newBroadcast[T](value_ : T, isLocal: Boolean) = new TreeBroadcast[T](value_, isLocal) - def stop() = MultiTracker.stop + def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new TreeBroadcast[T](value_, isLocal, id) + + def stop() { MultiTracker.stop() } } diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala index 344888919a..577101e3c3 100644 --- a/core/src/main/scala/spark/deploy/Command.scala +++ b/core/src/main/scala/spark/deploy/Command.scala @@ -2,7 +2,7 @@ package spark.deploy import scala.collection.Map -case class Command( +private[spark] case class Command( mainClass: String, arguments: Seq[String], environment: Map[String, String]) { diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 141bbe4d57..d2b63d6e0d 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -7,13 +7,15 @@ import scala.collection.immutable.List import scala.collection.mutable.HashMap -sealed trait DeployMessage extends Serializable +private[spark] sealed trait DeployMessage extends Serializable // Worker to Master +private[spark] case class RegisterWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int) extends DeployMessage +private[spark] case class ExecutorStateChanged( jobId: String, execId: Int, @@ -23,11 +25,11 @@ case class ExecutorStateChanged( // Master to Worker -case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage -case class RegisterWorkerFailed(message: String) extends DeployMessage -case class KillExecutor(jobId: String, execId: Int) extends DeployMessage +private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage +private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage +private[spark] case class KillExecutor(jobId: String, execId: Int) extends DeployMessage -case class LaunchExecutor( +private[spark] case class LaunchExecutor( jobId: String, execId: Int, jobDesc: JobDescription, @@ -38,33 +40,42 @@ case class LaunchExecutor( // Client to Master -case class RegisterJob(jobDescription: JobDescription) extends DeployMessage +private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage // Master to Client +private[spark] case class RegisteredJob(jobId: String) extends DeployMessage + +private[spark] case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) + +private[spark] case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String]) + +private[spark] case class JobKilled(message: String) // Internal message in Client -case object StopClient +private[spark] case object StopClient // MasterWebUI To Master -case object RequestMasterState +private[spark] case object RequestMasterState // Master to MasterWebUI +private[spark] case class MasterState(uri : String, workers: List[WorkerInfo], activeJobs: List[JobInfo], completedJobs: List[JobInfo]) // WorkerWebUI to Worker -case object RequestWorkerState +private[spark] case object RequestWorkerState // Worker to WorkerWebUI +private[spark] case class WorkerState(uri: String, workerId: String, executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String)
\ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index d6ff1c54ca..5dc0c54552 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,6 +1,6 @@ package spark.deploy -object ExecutorState +private[spark] object ExecutorState extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value diff --git a/core/src/main/scala/spark/deploy/JobDescription.scala b/core/src/main/scala/spark/deploy/JobDescription.scala index 8ae77b1038..20879c5f11 100644 --- a/core/src/main/scala/spark/deploy/JobDescription.scala +++ b/core/src/main/scala/spark/deploy/JobDescription.scala @@ -1,6 +1,6 @@ package spark.deploy -class JobDescription( +private[spark] class JobDescription( val name: String, val cores: Int, val memoryPerSlave: Int, diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala new file mode 100644 index 0000000000..8b2a71add5 --- /dev/null +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -0,0 +1,58 @@ +package spark.deploy + +import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} + +import spark.deploy.worker.Worker +import spark.deploy.master.Master +import spark.util.AkkaUtils +import spark.{Logging, Utils} + +import scala.collection.mutable.ArrayBuffer + +private[spark] +class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) extends Logging { + + val localIpAddress = Utils.localIpAddress + + var masterActor : ActorRef = _ + var masterActorSystem : ActorSystem = _ + var masterPort : Int = _ + var masterUrl : String = _ + + val slaveActorSystems = ArrayBuffer[ActorSystem]() + val slaveActors = ArrayBuffer[ActorRef]() + + def start() : String = { + logInfo("Starting a local Spark cluster with " + numSlaves + " slaves.") + + /* Start the Master */ + val (actorSystem, masterPort) = AkkaUtils.createActorSystem("sparkMaster", localIpAddress, 0) + masterActorSystem = actorSystem + masterUrl = "spark://" + localIpAddress + ":" + masterPort + val actor = masterActorSystem.actorOf( + Props(new Master(localIpAddress, masterPort, 0)), name = "Master") + masterActor = actor + + /* Start the Slaves */ + for (slaveNum <- 1 to numSlaves) { + val (actorSystem, boundPort) = + AkkaUtils.createActorSystem("sparkWorker" + slaveNum, localIpAddress, 0) + slaveActorSystems += actorSystem + val actor = actorSystem.actorOf( + Props(new Worker(localIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), + name = "Worker") + slaveActors += actor + } + + return masterUrl + } + + def stop() { + logInfo("Shutting down local Spark cluster.") + // Stop the slaves before the master so they don't get upset that it disconnected + slaveActorSystems.foreach(_.shutdown()) + slaveActorSystems.foreach(_.awaitTermination()) + masterActorSystem.shutdown() + masterActorSystem.awaitTermination() + } +} diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index c7fa8a3874..b1b72a3a1f 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -16,7 +16,7 @@ import akka.dispatch.Await * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description, * and a listener for job events, and calls back the listener when various events occur. */ -class Client( +private[spark] class Client( actorSystem: ActorSystem, masterUrl: String, jobDescription: JobDescription, @@ -42,7 +42,6 @@ class Client( val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort) try { master = context.actorFor(akkaUrl) - //master ! RegisterWorker(ip, port, cores, memory) master ! RegisterJob(jobDescription) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index 7d23baff32..a8fa982085 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -7,7 +7,7 @@ package spark.deploy.client * * Users of this API should *not* block inside the callback methods. */ -trait ClientListener { +private[spark] trait ClientListener { def connected(jobId: String): Unit def disconnected(): Unit diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index df9a36c7fe..bf0e7428ba 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -4,7 +4,7 @@ import spark.util.AkkaUtils import spark.{Logging, Utils} import spark.deploy.{Command, JobDescription} -object TestClient { +private[spark] object TestClient { class TestListener extends ClientListener with Logging { def connected(id: String) { diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala index 2e40e10d18..0e46db2272 100644 --- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala @@ -1,6 +1,6 @@ package spark.deploy.client -object TestExecutor { +private[spark] object TestExecutor { def main(args: Array[String]) { println("Hello world!") while (true) { diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala index 335e00958c..1db2c32633 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala @@ -2,7 +2,7 @@ package spark.deploy.master import spark.deploy.ExecutorState -class ExecutorInfo( +private[spark] class ExecutorInfo( val id: Int, val job: JobInfo, val worker: WorkerInfo, diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/JobInfo.scala index 31d48b82b9..8795c09cc1 100644 --- a/core/src/main/scala/spark/deploy/master/JobInfo.scala +++ b/core/src/main/scala/spark/deploy/master/JobInfo.scala @@ -5,6 +5,7 @@ import java.util.Date import akka.actor.ActorRef import scala.collection.mutable +private[spark] class JobInfo(val id: String, val desc: JobDescription, val submitDate: Date, val actor: ActorRef) { var state = JobState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -31,4 +32,13 @@ class JobInfo(val id: String, val desc: JobDescription, val submitDate: Date, va } def coresLeft: Int = desc.cores - coresGranted + + private var _retryCount = 0 + + def retryCount = _retryCount + + def incrementRetryCount = { + _retryCount += 1 + _retryCount + } } diff --git a/core/src/main/scala/spark/deploy/master/JobState.scala b/core/src/main/scala/spark/deploy/master/JobState.scala index 50b0c6f95b..2b70cf0191 100644 --- a/core/src/main/scala/spark/deploy/master/JobState.scala +++ b/core/src/main/scala/spark/deploy/master/JobState.scala @@ -1,7 +1,9 @@ package spark.deploy.master -object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { +private[spark] object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { type JobState = Value val WAITING, RUNNING, FINISHED, FAILED = Value + + val MAX_NUM_RETRY = 10 } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index c98dddea7b..6010f7cff2 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -1,21 +1,20 @@ package spark.deploy.master -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} - import akka.actor._ -import spark.{Logging, Utils} -import spark.util.AkkaUtils +import akka.actor.Terminated +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} + import java.text.SimpleDateFormat import java.util.Date -import akka.remote.RemoteClientLifeCycleEvent + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + import spark.deploy._ -import akka.remote.RemoteClientShutdown -import akka.remote.RemoteClientDisconnected -import spark.deploy.RegisterWorker -import spark.deploy.RegisterWorkerFailed -import akka.actor.Terminated +import spark.{Logging, SparkException, Utils} +import spark.util.AkkaUtils -class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { + +private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For job IDs var nextJobNumber = 0 @@ -81,12 +80,22 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { exec.state = state exec.job.actor ! ExecutorUpdated(execId, state, message) if (ExecutorState.isFinished(state)) { + val jobInfo = idToJob(jobId) // Remove this executor from the worker and job logInfo("Removing executor " + exec.fullId + " because it is " + state) - idToJob(jobId).removeExecutor(exec) + jobInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) - // TODO: the worker would probably want to restart the executor a few times - schedule() + + // Only retry certain number of times so we don't go into an infinite loop. + if (jobInfo.incrementRetryCount <= JobState.MAX_NUM_RETRY) { + schedule() + } else { + val e = new SparkException("Job %s wth ID %s failed %d times.".format( + jobInfo.desc.name, jobInfo.id, jobInfo.retryCount)) + logError(e.getMessage, e) + throw e + //System.exit(1) + } } } case None => @@ -112,7 +121,7 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { addressToWorker.get(address).foreach(removeWorker) addressToJob.get(address).foreach(removeJob) } - + case RequestMasterState => { sender ! MasterState(ip + ":" + port, workers.toList, jobs.toList, completedJobs.toList) } @@ -203,7 +212,7 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { } } -object Master { +private[spark] object Master { def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port) diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index 0f7a92bdd0..1b1c3dd0ad 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -6,7 +6,7 @@ import spark.Utils /** * Command-line parser for the master. */ -class MasterArguments(args: Array[String]) { +private[spark] class MasterArguments(args: Array[String]) { var ip = Utils.localIpAddress() var port = 7077 var webUiPort = 8080 @@ -51,7 +51,7 @@ class MasterArguments(args: Array[String]) { */ def printUsageAndExit(exitCode: Int) { System.err.println( - "Usage: spark-master [options]\n" + + "Usage: Master [options]\n" + "\n" + "Options:\n" + " -i IP, --ip IP IP address or DNS name to listen on\n" + diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index f03c0a0229..700a41c770 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,6 +10,7 @@ import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ import spark.deploy._ +private[spark] class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" @@ -22,7 +23,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct completeWith { val future = master ? RequestMasterState future.map { - masterState => masterui.html.index.render(masterState.asInstanceOf[MasterState]) + masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) } } } ~ @@ -36,7 +37,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct // A bit ugly an inefficient, but we won't have a number of jobs // so large that it will make a significant difference. (masterState.activeJobs ::: masterState.completedJobs).find(_.id == jobId) match { - case Some(job) => masterui.html.job_details.render(job) + case Some(job) => spark.deploy.master.html.job_details.render(job) case _ => null } } diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 59474a0945..16b3f9b653 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -3,7 +3,7 @@ package spark.deploy.master import akka.actor.ActorRef import scala.collection.mutable -class WorkerInfo( +private[spark] class WorkerInfo( val id: String, val host: String, val port: Int, diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 3e24380810..73722a82e0 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -13,7 +13,7 @@ import spark.deploy.ExecutorStateChanged /** * Manages the execution of one executor process. */ -class ExecutorRunner( +private[spark] class ExecutorRunner( val jobId: String, val execId: Int, val jobDesc: JobDescription, @@ -29,12 +29,25 @@ class ExecutorRunner( val fullId = jobId + "/" + execId var workerThread: Thread = null var process: Process = null + var shutdownHook: Thread = null def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } workerThread.start() + + // Shutdown hook that kills actors on shutdown. + shutdownHook = new Thread() { + override def run() { + if (process != null) { + logInfo("Shutdown hook killing child process.") + process.destroy() + process.waitFor() + } + } + } + Runtime.getRuntime.addShutdownHook(shutdownHook) } /** Stop this executor runner, including killing the process it launched */ @@ -45,40 +58,10 @@ class ExecutorRunner( if (process != null) { logInfo("Killing process!") process.destroy() + process.waitFor() } worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None) - } - } - - /** - * Download a file requested by the executor. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. - */ - def fetchFile(url: String, targetDir: File) { - val filename = url.split("/").last - val targetFile = new File(targetDir, filename) - if (url.startsWith("http://") || url.startsWith("https://") || url.startsWith("ftp://")) { - // Use the java.net library to fetch it - logInfo("Fetching " + url + " to " + targetFile) - val in = new URL(url).openStream() - val out = new FileOutputStream(targetFile) - Utils.copyStream(in, out, true) - } else { - // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others - val uri = new URI(url) - val conf = new Configuration() - val fs = FileSystem.get(uri, conf) - val in = fs.open(new Path(uri)) - val out = new FileOutputStream(targetFile) - Utils.copyStream(in, out, true) - } - // Decompress the file if it's a .tar or .tar.gz - if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xzf", filename), targetDir) - } else if (filename.endsWith(".tar")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xf", filename), targetDir) + Runtime.getRuntime.removeShutdownHook(shutdownHook) } } @@ -92,7 +75,8 @@ class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = jobDesc.command - val runScript = new File(sparkHome, "run").getCanonicalPath + val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run"; + val runScript = new File(sparkHome, script).getCanonicalPath Seq(runScript, command.mainClass) ++ command.arguments.map(substituteVariables) } @@ -101,7 +85,12 @@ class ExecutorRunner( val out = new FileOutputStream(file) new Thread("redirect output to " + file) { override def run() { - Utils.copyStream(in, out, true) + try { + Utils.copyStream(in, out, true) + } catch { + case e: IOException => + logInfo("Redirection to " + file + " closed: " + e.getMessage) + } } }.start() } @@ -131,6 +120,9 @@ class ExecutorRunner( } env.put("SPARK_CORES", cores.toString) env.put("SPARK_MEMORY", memory.toString) + // In case we are running this from within the Spark Shell + // so we are not creating a parent process. + env.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() // Redirect its stdout and stderr to files diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0a80463c0b..474c9364fd 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -16,7 +16,14 @@ import spark.deploy.RegisterWorkerFailed import akka.actor.Terminated import java.io.File -class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, masterUrl: String) +private[spark] class Worker( + ip: String, + port: Int, + webUiPort: Int, + cores: Int, + memory: Int, + masterUrl: String, + workDirPath: String = null) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs @@ -37,7 +44,11 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas def memoryFree: Int = memory - memoryUsed def createWorkDir() { - workDir = new File(sparkHome, "work") + workDir = if (workDirPath != null) { + new File(workDirPath) + } else { + new File(sparkHome, "work") + } try { if (!workDir.exists() && !workDir.mkdirs()) { logError("Failed to create work directory " + workDir) @@ -153,14 +164,19 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas def generateWorkerId(): String = { "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), ip, port) } + + override def postStop() { + executors.values.foreach(_.kill()) + } } -object Worker { +private[spark] object Worker { def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port) val actor = actorSystem.actorOf( - Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory, args.master)), + Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory, + args.master, args.workDir)), name = "Worker") actorSystem.awaitTermination() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 1efe8304ea..60dc107a4c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -8,13 +8,14 @@ import java.lang.management.ManagementFactory /** * Command-line parser for the master. */ -class WorkerArguments(args: Array[String]) { +private[spark] class WorkerArguments(args: Array[String]) { var ip = Utils.localIpAddress() var port = 0 var webUiPort = 8081 var cores = inferDefaultCores() var memory = inferDefaultMemory() var master: String = null + var workDir: String = null // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { @@ -29,6 +30,9 @@ class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } + if (System.getenv("SPARK_WORKER_DIR") != null) { + workDir = System.getenv("SPARK_WORKER_DIR") + } parse(args.toList) @@ -49,6 +53,10 @@ class WorkerArguments(args: Array[String]) { memory = value parse(tail) + case ("--work-dir" | "-d") :: value :: tail => + workDir = value + parse(tail) + case "--webui-port" :: IntParam(value) :: tail => webUiPort = value parse(tail) @@ -77,13 +85,14 @@ class WorkerArguments(args: Array[String]) { */ def printUsageAndExit(exitCode: Int) { System.err.println( - "Usage: spark-worker [options] <master>\n" + + "Usage: Worker [options] <master>\n" + "\n" + "Master must be a URL of the form spark://hostname:port\n" + "\n" + "Options:\n" + " -c CORES, --cores CORES Number of cores to use\n" + " -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" + + " -d DIR, --work-dir DIR Directory to run jobs in (default: SPARK_HOME/work)\n" + " -i IP, --ip IP IP address or DNS name to listen on\n" + " -p PORT, --port PORT Port to listen on (default: random)\n" + " --webui-port PORT Port for web UI (default: 8081)") diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 58a05e1a38..d06f4884ee 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -9,6 +9,7 @@ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ import spark.deploy.{WorkerState, RequestWorkerState} +private[spark] class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" @@ -21,7 +22,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct completeWith{ val future = worker ? RequestWorkerState future.map { workerState => - workerui.html.index(workerState.asInstanceOf[WorkerState]) + spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) } } } ~ diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index dba209ac27..6ecf9fa8da 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,10 +1,12 @@ package spark.executor import java.io.{File, FileOutputStream} -import java.net.{URL, URLClassLoader} +import java.net.{URI, URL, URLClassLoader} import java.util.concurrent._ -import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.fs.FileUtil + +import scala.collection.mutable.{ArrayBuffer, Map, HashMap} import spark.broadcast._ import spark.scheduler._ @@ -14,11 +16,16 @@ import java.nio.ByteBuffer /** * The Mesos executor for Spark. */ -class Executor extends Logging { - var classLoader: ClassLoader = null +private[spark] class Executor extends Logging { + var urlClassLoader : ExecutorURLClassLoader = null var threadPool: ExecutorService = null var env: SparkEnv = null + // Application dependencies (added through SparkContext) that we've fetched so far on this node. + // Each map holds the master's timestamp for the version of that file or JAR we got. + val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() + val currentJars: HashMap[String, Long] = new HashMap[String, Long]() + val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) initLogging() @@ -32,14 +39,14 @@ class Executor extends Logging { System.setProperty(key, value) } + // Create our ClassLoader and set it on this thread + urlClassLoader = createClassLoader() + Thread.currentThread.setContextClassLoader(urlClassLoader) + // Initialize Spark environment (using system properties read above) env = SparkEnv.createFromSystemProperties(slaveHostname, 0, false, false) SparkEnv.set(env) - // Create our ClassLoader (using spark properties) and set it on this thread - classLoader = createClassLoader() - Thread.currentThread.setContextClassLoader(classLoader) - // Start worker thread pool threadPool = new ThreadPoolExecutor( 1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable]) @@ -54,15 +61,16 @@ class Executor extends Logging { override def run() { SparkEnv.set(env) - Thread.currentThread.setContextClassLoader(classLoader) + Thread.currentThread.setContextClassLoader(urlClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo("Running task ID " + taskId) context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) try { SparkEnv.set(env) - Thread.currentThread.setContextClassLoader(classLoader) Accumulators.clear() - val task = ser.deserialize[Task[Any]](serializedTask, classLoader) + val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) + updateDependencies(taskFiles, taskJars) + val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) logInfo("Its generation is " + task.generation) env.mapOutputTracker.updateGeneration(task.generation) val value = task.run(taskId.toInt) @@ -96,25 +104,15 @@ class Executor extends Logging { * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): ClassLoader = { + private def createClassLoader(): ExecutorURLClassLoader = { var loader = this.getClass.getClassLoader - // If any JAR URIs are given through spark.jar.uris, fetch them to the - // current directory and put them all on the classpath. We assume that - // each URL has a unique file name so that no local filenames will clash - // in this process. This is guaranteed by ClusterScheduler. - val uris = System.getProperty("spark.jar.uris", "") - val localFiles = ArrayBuffer[String]() - for (uri <- uris.split(",").filter(_.size > 0)) { - val url = new URL(uri) - val filename = url.getPath.split("/").last - downloadFile(url, filename) - localFiles += filename - } - if (localFiles.size > 0) { - val urls = localFiles.map(f => new File(f).toURI.toURL).toArray - loader = new URLClassLoader(urls, loader) - } + // For each of the jars in the jarSet, add them to the class loader. + // We assume each of the files has already been fetched. + val urls = currentJars.keySet.map { uri => + new File(uri.split("/").last).toURI.toURL + }.toArray + loader = new URLClassLoader(urls, loader) // If the REPL is in use, add another ClassLoader that will read // new classes defined by the REPL as the user types code @@ -133,13 +131,31 @@ class Executor extends Logging { } } - return loader + return new ExecutorURLClassLoader(Array(), loader) } - // Download a file from a given URL to the local filesystem - private def downloadFile(url: URL, localPath: String) { - val in = url.openStream() - val out = new FileOutputStream(localPath) - Utils.copyStream(in, out, true) + /** + * Download any missing dependencies if we receive a new set of files and JARs from the + * SparkContext. Also adds any new JARs we fetched to the class loader. + */ + private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { + // Fetch missing dependencies + for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name) + Utils.fetchFile(name, new File(".")) + currentFiles(name) = timestamp + } + for ((name, timestamp) <- newJars if currentFiles.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name) + Utils.fetchFile(name, new File(".")) + currentJars(name) = timestamp + // Add it to our class loader + val localName = name.split("/").last + val url = new File(".", localName).toURI.toURL + if (!urlClassLoader.getURLs.contains(url)) { + logInfo("Adding " + url + " to class loader") + urlClassLoader.addURL(url) + } + } } } diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala index 24c8776f31..e97e509700 100644 --- a/core/src/main/scala/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala @@ -6,6 +6,6 @@ import spark.TaskState.TaskState /** * A pluggable interface used by the Executor to send updates to the cluster scheduler. */ -trait ExecutorBackend { +private[spark] trait ExecutorBackend { def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) } diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala new file mode 100644 index 0000000000..5beb4d049e --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala @@ -0,0 +1,14 @@ +package spark.executor + +import java.net.{URLClassLoader, URL} + +/** + * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. + */ +private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends URLClassLoader(urls, parent) { + + override def addURL(url: URL) { + super.addURL(url) + } +} diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 50f4e41ede..eeab3959c6 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -8,7 +8,7 @@ import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState -class MesosExecutorBackend(executor: Executor) +private[spark] class MesosExecutorBackend(executor: Executor) extends MesosExecutor with ExecutorBackend with Logging { @@ -59,7 +59,7 @@ class MesosExecutorBackend(executor: Executor) /** * Entry point for Mesos executor. */ -object MesosExecutorBackend { +private[spark] object MesosExecutorBackend { def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 26b163de0a..915f71ba9f 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -14,7 +14,7 @@ import spark.scheduler.cluster.RegisterSlaveFailed import spark.scheduler.cluster.RegisterSlave -class StandaloneExecutorBackend( +private[spark] class StandaloneExecutorBackend( executor: Executor, masterUrl: String, slaveId: String, @@ -62,7 +62,7 @@ class StandaloneExecutorBackend( } } -object StandaloneExecutorBackend { +private[spark] object StandaloneExecutorBackend { def run(masterUrl: String, slaveId: String, hostname: String, cores: Int) { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index da8aff9dd5..80262ab7b4 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -11,6 +11,7 @@ import java.nio.channels.spi._ import java.net._ +private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector) extends Logging { channel.configureBlocking(false) @@ -23,8 +24,8 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex var onExceptionCallback: (Connection, Exception) => Unit = null var onKeyInterestChangeCallback: (Connection, Int) => Unit = null - lazy val remoteAddress = getRemoteAddress() - lazy val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress) + val remoteAddress = getRemoteAddress() + val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress) def key() = channel.keyFor(selector) @@ -39,7 +40,10 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex } def close() { - key.cancel() + val k = key() + if (k != null) { + k.cancel() + } channel.close() callOnCloseCallback() } @@ -99,7 +103,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex } -class SendingConnection(val address: InetSocketAddress, selector_ : Selector) +private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector) extends Connection(SocketChannel.open, selector_) { class Outbox(fair: Int = 0) { @@ -134,9 +138,12 @@ extends Connection(SocketChannel.open, selector_) { if (!message.started) logDebug("Starting to send [" + message + "]") message.started = true return chunk + } else { + /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/ + message.finishTime = System.currentTimeMillis + logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + + "] in " + message.timeTaken ) } - /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/ - logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "] in " + message.timeTaken ) } } None @@ -159,10 +166,11 @@ extends Connection(SocketChannel.open, selector_) { } logTrace("Sending chunk from [" + message+ "] to [" + remoteConnectionManagerId + "]") return chunk - } - /*messages -= message*/ - message.finishTime = System.currentTimeMillis - logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "] in " + message.timeTaken ) + } else { + message.finishTime = System.currentTimeMillis + logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + + "] in " + message.timeTaken ) + } } } None @@ -216,7 +224,7 @@ extends Connection(SocketChannel.open, selector_) { while(true) { if (currentBuffers.size == 0) { outbox.synchronized { - outbox.getChunk match { + outbox.getChunk() match { case Some(chunk) => { currentBuffers ++= chunk.buffers } @@ -252,7 +260,7 @@ extends Connection(SocketChannel.open, selector_) { } -class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) +private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) extends Connection(channel_, selector_) { class Inbox() { diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 0e764fff81..dec0df25b4 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -16,18 +16,19 @@ import scala.collection.mutable.ArrayBuffer import akka.dispatch.{Await, Promise, ExecutionContext, Future} import akka.util.Duration +import akka.util.duration._ -case class ConnectionManagerId(host: String, port: Int) { +private[spark] case class ConnectionManagerId(host: String, port: Int) { def toSocketAddress() = new InetSocketAddress(host, port) } -object ConnectionManagerId { +private[spark] object ConnectionManagerId { def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) } } -class ConnectionManager(port: Int) extends Logging { +private[spark] class ConnectionManager(port: Int) extends Logging { class MessageStatus( val message: Message, @@ -348,7 +349,7 @@ class ConnectionManager(port: Int) extends Logging { } -object ConnectionManager { +private[spark] object ConnectionManager { def main(args: Array[String]) { @@ -403,7 +404,10 @@ object ConnectionManager { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) - }).foreach(f => {if (!f().isDefined) println("Failed")}) + }).foreach(f => { + val g = Await.result(f, 1 second) + if (!g.isDefined) println("Failed") + }) val finishTime = System.currentTimeMillis val mb = size * count / 1024.0 / 1024.0 @@ -430,7 +434,10 @@ object ConnectionManager { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffers(count - 1 - i).duplicate) manager.sendMessageReliably(manager.id, bufferMessage) - }).foreach(f => {if (!f().isDefined) println("Failed")}) + }).foreach(f => { + val g = Await.result(f, 1 second) + if (!g.isDefined) println("Failed") + }) val finishTime = System.currentTimeMillis val ms = finishTime - startTime @@ -457,7 +464,10 @@ object ConnectionManager { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) - }).foreach(f => {if (!f().isDefined) println("Failed")}) + }).foreach(f => { + val g = Await.result(f, 1 second) + if (!g.isDefined) println("Failed") + }) val finishTime = System.currentTimeMillis Thread.sleep(1000) val mb = size * count / 1024.0 / 1024.0 diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 5d21bb793f..47ceaf3c07 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -8,7 +8,10 @@ import scala.io.Source import java.nio.ByteBuffer import java.net.InetAddress -object ConnectionManagerTest extends Logging{ +import akka.dispatch.Await +import akka.util.duration._ + +private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { if (args.length < 2) { println("Usage: ConnectionManagerTest <mesos cluster> <slaves file>") @@ -53,7 +56,7 @@ object ConnectionManagerTest extends Logging{ logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) }) - val results = futures.map(f => f()) + val results = futures.map(f => Await.result(f, 1.second)) val finishTime = System.currentTimeMillis Thread.sleep(5000) diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index 2e85803679..525751b5bf 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -7,8 +7,9 @@ import scala.collection.mutable.ArrayBuffer import java.nio.ByteBuffer import java.net.InetAddress import java.net.InetSocketAddress +import storage.BlockManager -class MessageChunkHeader( +private[spark] class MessageChunkHeader( val typ: Long, val id: Int, val totalSize: Int, @@ -36,7 +37,7 @@ class MessageChunkHeader( " and sizes " + totalSize + " / " + chunkSize + " bytes" } -class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { +private[spark] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { val size = if (buffer == null) 0 else buffer.remaining lazy val buffers = { val ab = new ArrayBuffer[ByteBuffer]() @@ -50,7 +51,7 @@ class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { override def toString = "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" } -abstract class Message(val typ: Long, val id: Int) { +private[spark] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null var started = false var startTime = -1L @@ -64,10 +65,10 @@ abstract class Message(val typ: Long, val id: Int) { def timeTaken(): String = (finishTime - startTime).toString + " ms" - override def toString = "" + this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" + override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" } -class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) +private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) extends Message(Message.BUFFER_MESSAGE, id_) { val initialSize = currentSize() @@ -97,10 +98,11 @@ extends Message(Message.BUFFER_MESSAGE, id_) { while(!buffers.isEmpty) { val buffer = buffers(0) if (buffer.remaining == 0) { + BlockManager.dispose(buffer) buffers -= buffer } else { val newBuffer = if (buffer.remaining <= maxChunkSize) { - buffer.duplicate + buffer.duplicate() } else { buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer] } @@ -147,11 +149,10 @@ extends Message(Message.BUFFER_MESSAGE, id_) { } else { "BufferMessage(id = " + id + ", size = " + size + ")" } - } } -object MessageChunkHeader { +private[spark] object MessageChunkHeader { val HEADER_SIZE = 40 def create(buffer: ByteBuffer): MessageChunkHeader = { @@ -172,7 +173,7 @@ object MessageChunkHeader { } } -object Message { +private[spark] object Message { val BUFFER_MESSAGE = 1111111111L var lastId = 1 diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala index e1ba7c06c0..a174d5f403 100644 --- a/core/src/main/scala/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/spark/network/ReceiverTest.scala @@ -3,7 +3,7 @@ package spark.network import java.nio.ByteBuffer import java.net.InetAddress -object ReceiverTest { +private[spark] object ReceiverTest { def main(args: Array[String]) { val manager = new ConnectionManager(9999) diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala index 4ab6dd3414..a4ff69e4d2 100644 --- a/core/src/main/scala/spark/network/SenderTest.scala +++ b/core/src/main/scala/spark/network/SenderTest.scala @@ -3,7 +3,7 @@ package spark.network import java.nio.ByteBuffer import java.net.InetAddress -object SenderTest { +private[spark] object SenderTest { def main(args: Array[String]) { diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index e6535836ab..42f46e06ed 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -12,7 +12,7 @@ import spark.scheduler.JobListener * a result of type U for each partition, and that the action returns a partial or complete result * of type R. Note that the type R must *include* any error bars on it (e.g. see BoundedInt). */ -class ApproximateActionListener[T, U, R]( +private[spark] class ApproximateActionListener[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala index 4772e43ef0..75713b2eaa 100644 --- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala +++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala @@ -4,7 +4,7 @@ package spark.partial * An object that computes a function incrementally by merging in results of type U from multiple * tasks. Allows partial evaluation at any point by calling currentResult(). */ -trait ApproximateEvaluator[U, R] { +private[spark] trait ApproximateEvaluator[U, R] { def merge(outputId: Int, taskResult: U): Unit def currentResult(): R } diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/spark/partial/BoundedDouble.scala index 463c33d6e2..8bedd75182 100644 --- a/core/src/main/scala/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/spark/partial/BoundedDouble.scala @@ -3,6 +3,7 @@ package spark.partial /** * A Double with error bars on it. */ +private[spark] class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala index 1bc90d6b39..daf2c5170c 100644 --- a/core/src/main/scala/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/spark/partial/CountEvaluator.scala @@ -8,7 +8,7 @@ import cern.jet.stat.Probability * TODO: There's currently a lot of shared code between this and GroupedCountEvaluator. It might * be best to make this a special case of GroupedCountEvaluator with one group. */ -class CountEvaluator(totalOutputs: Int, confidence: Double) +private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[Long, BoundedDouble] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala index 3e631c0efc..01fbb8a11b 100644 --- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala @@ -14,7 +14,7 @@ import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} /** * An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval. */ -class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) +private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala index 2a9ccba205..c622df5220 100644 --- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala @@ -12,7 +12,7 @@ import spark.util.StatCounter /** * An ApproximateEvaluator for means by key. Returns a map of key to confidence interval. */ -class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double) +private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala index 6a2ec7a7bd..20fa55cff2 100644 --- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala @@ -12,7 +12,7 @@ import spark.util.StatCounter /** * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval. */ -class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double) +private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala index b8c7cb8863..762c85400d 100644 --- a/core/src/main/scala/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala @@ -7,7 +7,7 @@ import spark.util.StatCounter /** * An ApproximateEvaluator for means. */ -class MeanEvaluator(totalOutputs: Int, confidence: Double) +private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/spark/partial/PartialResult.scala index 200ed4ea1e..beafbf67c3 100644 --- a/core/src/main/scala/spark/partial/PartialResult.scala +++ b/core/src/main/scala/spark/partial/PartialResult.scala @@ -1,6 +1,6 @@ package spark.partial -class PartialResult[R](initialVal: R, isFinal: Boolean) { +private[spark] class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None private var completionHandler: Option[R => Unit] = None diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala index 6263ee3518..443abba5cd 100644 --- a/core/src/main/scala/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/spark/partial/StudentTCacher.scala @@ -7,7 +7,7 @@ import cern.jet.stat.Probability * and various sample sizes. This is used by the MeanEvaluator to efficiently calculate * confidence intervals for many keys. */ -class StudentTCacher(confidence: Double) { +private[spark] class StudentTCacher(confidence: Double) { val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation val normalApprox = Probability.normalInverse(1 - (1 - confidence) / 2) val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0) diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala index 0357a6bff8..58fb60f441 100644 --- a/core/src/main/scala/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/spark/partial/SumEvaluator.scala @@ -9,7 +9,7 @@ import spark.util.StatCounter * together, then uses the formula for the variance of two independent random variables to get * a variance for the result and compute a confidence interval. */ -class SumEvaluator(totalOutputs: Int, confidence: Double) +private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { var outputsMerged = 0 diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 0ecff9ce77..5a4e9a582d 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -5,11 +5,12 @@ import spark.TaskContext /** * Tracks information about an active job in the DAGScheduler. */ -class ActiveJob( +private[spark] class ActiveJob( val runId: Int, val finalStage: Stage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], + val callSite: String, val listener: JobListener) { val numPartitions = partitions.length diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7472971b5..9b666ed181 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -21,6 +21,7 @@ import spark.storage.BlockManagerId * schedule to run the job. Subclasses only need to implement the code to send a task to the cluster * and to report fetch failures (the submitTasks method, and code to add CompletionEvents). */ +private[spark] class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Logging { taskSched.setListener(this) @@ -38,6 +39,11 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with eventQueue.put(HostLost(host)) } + // Called by TaskScheduler to cancel an entier TaskSet due to repeated failures. + override def taskSetFailed(taskSet: TaskSet, reason: String) { + eventQueue.put(TaskSetFailed(taskSet, reason)) + } + // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one // as more failure events come in @@ -116,7 +122,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_,_]], priority: Int): Stage = { // 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 splits is unknown - logInfo("Registering RDD " + rdd.id + ": " + rdd) + logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") cacheTracker.registerRDD(rdd.id, rdd.splits.size) if (shuffleDep != None) { mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size) @@ -139,7 +145,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with visited += r // Kind of ugly: need to register RDDs with the cache here since // we can't do it in its constructor because # of splits is unknown - logInfo("Registering parent RDD " + r.id + ": " + r) + logInfo("Registering parent RDD " + r.id + " (" + r.origin + ")") cacheTracker.registerRDD(r.id, r.splits.size) for (dep <- r.dependencies) { dep match { @@ -183,23 +189,25 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with missing.toList } - def runJob[T, U]( + def runJob[T, U: ClassManifest]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], + callSite: String, allowLocal: Boolean) - (implicit m: ClassManifest[U]): Array[U] = + : Array[U] = { if (partitions.size == 0) { return new Array[U](0) } val waiter = new JobWaiter(partitions.size) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, waiter)) + eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter)) waiter.getResult() match { case JobSucceeded(results: Seq[_]) => return results.asInstanceOf[Seq[U]].toArray case JobFailed(exception: Exception) => + logInfo("Failed to run " + callSite) throw exception } } @@ -208,13 +216,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], - timeout: Long - ): PartialResult[R] = + callSite: String, + timeout: Long) + : PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.splits.size).toArray - eventQueue.put(JobSubmitted(rdd, func2, partitions, false, listener)) + eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener)) return listener.getResult() // Will throw an exception if the job fails } @@ -234,13 +243,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } event match { - case JobSubmitted(finalRDD, func, partitions, allowLocal, listener) => + case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) => val runId = nextRunId.getAndIncrement() val finalStage = newStage(finalRDD, None, runId) - val job = new ActiveJob(runId, finalStage, func, partitions, listener) + val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) updateCacheLocs() - logInfo("Got job " + job.runId + " with " + partitions.length + " output partitions") - logInfo("Final stage: " + finalStage) + logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + + " output partitions") + logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { @@ -258,6 +268,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with case completion: CompletionEvent => handleTaskCompletion(completion) + case TaskSetFailed(taskSet, reason) => + abortStage(idToStage(taskSet.stageId), reason) + case StopDAGScheduler => // Cancel any active jobs for (job <- activeJobs) { @@ -329,7 +342,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing == Nil) { - logInfo("Submitting " + stage + ", which has no missing parents") + logInfo("Submitting " + stage + " (" + stage.origin + "), which has no missing parents") submitMissingTasks(stage) running += stage } else { @@ -416,7 +429,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with stage.addOutputLoc(smt.partition, bmAddress) } if (running.contains(stage) && pendingTasks(stage).isEmpty) { - logInfo(stage + " finished; looking for newly runnable stages") + logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages") running -= stage logInfo("running: " + running) logInfo("waiting: " + waiting) @@ -430,7 +443,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (stage.outputLocs.count(_ == Nil) != 0) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " because some of its tasks had failed: " + + logInfo("Resubmitting " + stage + " (" + stage.origin + + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) submitStage(stage) } else { @@ -444,6 +458,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with waiting --= newlyRunnable running ++= newlyRunnable for (stage <- newlyRunnable.sortBy(_.id)) { + logInfo("Submitting " + stage + " (" + stage.origin + "), which is now runnable") submitMissingTasks(stage) } } @@ -460,12 +475,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with running -= failedStage failed += failedStage // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " for resubmision due to a fetch failure") + logInfo("Marking " + failedStage + " (" + failedStage.origin + + ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) - logInfo("The failed fetch was from " + mapStage + "; marking it for resubmission") + logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin + + "); marking it for resubmission") failed += mapStage // Remember that a fetch failed now; this is used to resubmit the broken // stages later, after a small wait (to give other tasks the chance to fail) @@ -475,18 +492,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with handleHostLost(bmAddress.ip) } - case _ => - // Non-fetch failure -- probably a bug in the job, so bail out - // TODO: Cancel all tasks that are still running - resultStageToJob.get(stage) match { - case Some(job) => - val error = new SparkException("Task failed: " + task + ", reason: " + event.reason) - job.listener.jobFailed(error) - activeJobs -= job - resultStageToJob -= stage - case None => - logInfo("Ignoring result from " + task + " because its job has finished") - } + case other => + // Non-fetch failure -- probably a bug in user code; abort all jobs depending on this stage + abortStage(idToStage(task.stageId), task + " failed: " + other) } } @@ -509,6 +517,53 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with updateCacheLocs() } } + + /** + * Aborts all jobs depending on a particular Stage. This is called in response to a task set + * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. + */ + def abortStage(failedStage: Stage, reason: String) { + val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq + for (resultStage <- dependentStages) { + val job = resultStageToJob(resultStage) + job.listener.jobFailed(new SparkException("Job failed: " + reason)) + activeJobs -= job + resultStageToJob -= resultStage + } + if (dependentStages.isEmpty) { + logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") + } + } + + /** + * Return true if one of stage's ancestors is target. + */ + def stageDependsOn(stage: Stage, target: Stage): Boolean = { + if (stage == target) { + return true + } + val visitedRdds = new HashSet[RDD[_]] + val visitedStages = new HashSet[Stage] + def visit(rdd: RDD[_]) { + if (!visitedRdds(rdd)) { + visitedRdds += rdd + for (dep <- rdd.dependencies) { + dep match { + case shufDep: ShuffleDependency[_,_,_] => + val mapStage = getShuffleMapStage(shufDep, stage.priority) + if (!mapStage.isAvailable) { + visitedStages += mapStage + visit(mapStage.rdd) + } // Otherwise there's no need to follow the dependency back + case narrowDep: NarrowDependency[_] => + visit(narrowDep.rdd) + } + } + } + } + visit(stage.rdd) + visitedRdds.contains(target.rdd) + } def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { // If the partition is cached, return the cache locations diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 0fc73059c3..3422a21d9d 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -10,23 +10,26 @@ import spark._ * submitted) but there is a single "logic" thread that reads these events and takes decisions. * This greatly simplifies synchronization. */ -sealed trait DAGSchedulerEvent +private[spark] sealed trait DAGSchedulerEvent -case class JobSubmitted( +private[spark] case class JobSubmitted( finalRDD: RDD[_], func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], allowLocal: Boolean, + callSite: String, listener: JobListener) extends DAGSchedulerEvent -case class CompletionEvent( +private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]) extends DAGSchedulerEvent -case class HostLost(host: String) extends DAGSchedulerEvent +private[spark] case class HostLost(host: String) extends DAGSchedulerEvent -case object StopDAGScheduler extends DAGSchedulerEvent +private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent + +private[spark] case object StopDAGScheduler extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala index d4dd536a7d..f46b9d551d 100644 --- a/core/src/main/scala/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/spark/scheduler/JobListener.scala @@ -5,7 +5,7 @@ package spark.scheduler * DAGScheduler. The listener is notified each time a task succeeds, as well as if the whole * job fails (and no further taskSucceeded events will happen). */ -trait JobListener { +private[spark] trait JobListener { def taskSucceeded(index: Int, result: Any) def jobFailed(exception: Exception) } diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index 62b458eccb..c4a74e526f 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -3,7 +3,7 @@ package spark.scheduler /** * A result of a job in the DAGScheduler. */ -sealed trait JobResult +private[spark] sealed trait JobResult -case class JobSucceeded(results: Seq[_]) extends JobResult -case class JobFailed(exception: Exception) extends JobResult +private[spark] case class JobSucceeded(results: Seq[_]) extends JobResult +private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 4c2ae23051..b3d4feebe5 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer /** * An object that waits for a DAGScheduler job to complete. */ -class JobWaiter(totalTasks: Int) extends JobListener { +private[spark] class JobWaiter(totalTasks: Int) extends JobListener { private val taskResults = ArrayBuffer.fill[Any](totalTasks)(null) private var finishedTasks = 0 diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 090ced9d76..2ebd4075a2 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -2,7 +2,7 @@ package spark.scheduler import spark._ -class ResultTask[T, U]( +private[spark] class ResultTask[T, U]( stageId: Int, rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index a281ae94c5..966a5e173a 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -1,10 +1,10 @@ package spark.scheduler import java.io._ -import java.util.HashMap +import java.util.{HashMap => JHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream @@ -15,12 +15,12 @@ import com.ning.compress.lzf.LZFOutputStream import spark._ import spark.storage._ -object ShuffleMapTask { +private[spark] object ShuffleMapTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new HashMap[Int, Array[Byte]] + val serializedInfoCache = new JHashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_,_]): Array[Byte] = { synchronized { @@ -29,7 +29,8 @@ object ShuffleMapTask { return old } else { val out = new ByteArrayOutputStream - val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) + val ser = SparkEnv.get.closureSerializer.newInstance + val objOut = ser.serializeStream(new GZIPOutputStream(out)) objOut.writeObject(rdd) objOut.writeObject(dep) objOut.close() @@ -44,16 +45,22 @@ object ShuffleMapTask { synchronized { val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass) = - Class.forName(desc.getName, false, loader) - } + val ser = SparkEnv.get.closureSerializer.newInstance + val objIn = ser.deserializeStream(in) val rdd = objIn.readObject().asInstanceOf[RDD[_]] val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_,_]] return (rdd, dep) } } + // Since both the JarSet and FileSet have the same format this is used for both. + def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = { + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val objIn = new ObjectInputStream(in) + val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap + return (HashMap(set.toSeq: _*)) + } + def clearCache() { synchronized { serializedInfoCache.clear() @@ -61,7 +68,7 @@ object ShuffleMapTask { } } -class ShuffleMapTask( +private[spark] class ShuffleMapTask( stageId: Int, var rdd: RDD[_], var dep: ShuffleDependency[_,_,_], @@ -110,7 +117,7 @@ class ShuffleMapTask( val bucketIterators = if (aggregator.mapSideCombine) { // Apply combiners (map-side aggregation) to the map output. - val buckets = Array.tabulate(numOutputSplits)(_ => new HashMap[Any, Any]) + val buckets = Array.tabulate(numOutputSplits)(_ => new JHashMap[Any, Any]) for (elem <- rdd.iterator(split)) { val (k, v) = elem.asInstanceOf[(Any, Any)] val bucketId = partitioner.getPartition(k) @@ -125,7 +132,7 @@ class ShuffleMapTask( buckets.map(_.iterator) } else { // No combiners (no map-side aggregation). Simply partition the map output. - val buckets = Array.tabulate(numOutputSplits)(_ => new ArrayBuffer[(Any, Any)]) + val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) for (elem <- rdd.iterator(split)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = partitioner.getPartition(pair._1) @@ -137,11 +144,11 @@ class ShuffleMapTask( val ser = SparkEnv.get.serializer.newInstance() val blockManager = SparkEnv.get.blockManager for (i <- 0 until numOutputSplits) { - val blockId = "shuffleid_" + dep.shuffleId + "_" + partition + "_" + i + val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i // Get a scala iterator from java map val iter: Iterator[(Any, Any)] = bucketIterators(i) // TODO: This should probably be DISK_ONLY - blockManager.put(blockId, iter, StorageLevel.MEMORY_ONLY, false) + blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) } return SparkEnv.get.blockManager.blockManagerId diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index cd660c9085..803dd1b97d 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -19,7 +19,7 @@ import spark.storage.BlockManagerId * Each Stage also has a priority, which is (by default) based on the job it was submitted in. * This allows Stages from earlier jobs to be computed first or recovered faster on failure. */ -class Stage( +private[spark] class Stage( val id: Int, val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_,_]], // Output shuffle if stage is a map stage @@ -80,6 +80,8 @@ class Stage( return id } + def origin: String = rdd.origin + override def toString = "Stage " + id // + ": [RDD = " + rdd.id + ", isShuffle = " + isShuffleMap + "]" override def hashCode(): Int = id diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index f84d8d9c4f..d449ac67d6 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,11 +1,95 @@ package spark.scheduler +import scala.collection.mutable.{HashMap} +import spark.{SerializerInstance, Serializer, Utils} +import java.io.{DataInputStream, DataOutputStream, File} +import java.nio.ByteBuffer +import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream +import spark.util.ByteBufferInputStream +import scala.collection.mutable.HashMap + /** * A task to execute on a worker node. */ -abstract class Task[T](val stageId: Int) extends Serializable { +private[spark] abstract class Task[T](val stageId: Int) extends Serializable { def run(attemptId: Long): T def preferredLocations: Seq[String] = Nil var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. } + +/** + * Handles transmission of tasks and their dependencies, because this can be slightly tricky. We + * need to send the list of JARs and files added to the SparkContext with each task to ensure that + * worker nodes find out about it, but we can't make it part of the Task because the user's code in + * the task might depend on one of the JARs. Thus we serialize each task as multiple objects, by + * first writing out its dependencies. + */ +private[spark] object Task { + /** + * Serialize a task and the current app dependencies (files and JARs added to the SparkContext) + */ + def serializeWithDependencies( + task: Task[_], + currentFiles: HashMap[String, Long], + currentJars: HashMap[String, Long], + serializer: SerializerInstance) + : ByteBuffer = { + + val out = new FastByteArrayOutputStream(4096) + val dataOut = new DataOutputStream(out) + + // Write currentFiles + dataOut.writeInt(currentFiles.size) + for ((name, timestamp) <- currentFiles) { + dataOut.writeUTF(name) + dataOut.writeLong(timestamp) + } + + // Write currentJars + dataOut.writeInt(currentJars.size) + for ((name, timestamp) <- currentJars) { + dataOut.writeUTF(name) + dataOut.writeLong(timestamp) + } + + // Write the task itself and finish + dataOut.flush() + val taskBytes = serializer.serialize(task).array() + out.write(taskBytes) + out.trim() + ByteBuffer.wrap(out.array) + } + + /** + * Deserialize the list of dependencies in a task serialized with serializeWithDependencies, + * and return the task itself as a serialized ByteBuffer. The caller can then update its + * ClassLoaders and deserialize the task. + * + * @return (taskFiles, taskJars, taskBytes) + */ + def deserializeWithDependencies(serializedTask: ByteBuffer) + : (HashMap[String, Long], HashMap[String, Long], ByteBuffer) = { + + val in = new ByteBufferInputStream(serializedTask) + val dataIn = new DataInputStream(in) + + // Read task's files + val taskFiles = new HashMap[String, Long]() + val numFiles = dataIn.readInt() + for (i <- 0 until numFiles) { + taskFiles(dataIn.readUTF()) = dataIn.readLong() + } + + // Read task's JARs + val taskJars = new HashMap[String, Long]() + val numJars = dataIn.readInt() + for (i <- 0 until numJars) { + taskJars(dataIn.readUTF()) = dataIn.readLong() + } + + // Create a sub-buffer for the rest of the data, which is the serialized Task object + val subBuffer = serializedTask.slice() // ByteBufferInputStream will have read just up to task + (taskFiles, taskJars, subBuffer) + } +}
\ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 868ddb237c..9a54d0e854 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -7,6 +7,7 @@ import scala.collection.mutable.Map // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it +private[spark] class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Externalizable { def this() = this(null.asInstanceOf[T], null) diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index c35633d53c..d549b184b0 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -7,7 +7,7 @@ package spark.scheduler * are failures, and mitigating stragglers. They return events to the DAGScheduler through * the TaskSchedulerListener interface. */ -trait TaskScheduler { +private[spark] trait TaskScheduler { def start(): Unit // Disconnect from the cluster. diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index a647eec9e4..fa4de15d0d 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -7,10 +7,13 @@ import spark.TaskEndReason /** * Interface for getting events back from the TaskScheduler. */ -trait TaskSchedulerListener { +private[spark] trait TaskSchedulerListener { // A task has finished or failed. def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit // A node was lost from the cluster. def hostLost(host: String): Unit + + // The TaskScheduler wants to abort an entire task set. + def taskSetFailed(taskSet: TaskSet, reason: String): Unit } diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index 6f29dd2e9d..a3002ca477 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -4,6 +4,8 @@ package spark.scheduler * A set of tasks submitted together to the low-level TaskScheduler, usually representing * missing partitions of a particular stage. */ -class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int) { +private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int) { val id: String = stageId + "." + attempt + + override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 5b59479682..f5e852d203 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -16,7 +16,7 @@ import java.util.concurrent.atomic.AtomicLong * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call * start(), then submit task sets through the runTasks method. */ -class ClusterScheduler(sc: SparkContext) +private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler with Logging { @@ -60,7 +60,6 @@ class ClusterScheduler(sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - createJarServer() } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -115,6 +114,7 @@ class ClusterScheduler(sc: SparkContext) */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { synchronized { + SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { slaveIdToHost(o.slaveId) = o.hostname @@ -235,32 +235,7 @@ class ClusterScheduler(sc: SparkContext) } override def defaultParallelism() = backend.defaultParallelism() - - // Create a server for all the JARs added by the user to SparkContext. - // We first copy the JARs to a temp directory for easier server setup. - private def createJarServer() { - val jarDir = Utils.createTempDir() - logInfo("Temp directory for JARs: " + jarDir) - val filenames = ArrayBuffer[String]() - // Copy each JAR to a unique filename in the jarDir - for ((path, index) <- sc.jars.zipWithIndex) { - val file = new File(path) - if (file.exists) { - val filename = index + "_" + file.getName - Utils.copyFile(file, new File(jarDir, filename)) - filenames += filename - } - } - // Create the server - jarServer = new HttpServer(jarDir) - jarServer.start() - // Build up the jar URI list - val serverUri = jarServer.uri - jarUris = filenames.map(f => serverUri + "/" + f).mkString(",") - System.setProperty("spark.jar.uris", jarUris) - logInfo("JAR server started at " + serverUri) - } - + // Check for speculatable tasks in all our active jobs. def checkSpeculatableTasks() { var shouldRevive = false diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 897976c3f9..ddcd64d7c6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -5,7 +5,7 @@ package spark.scheduler.cluster * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as * machines become available and can launch tasks on them. */ -trait SchedulerBackend { +private[spark] trait SchedulerBackend { def start(): Unit def stop(): Unit def reviveOffers(): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala index e15d577a8b..96ebaa4601 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala @@ -1,3 +1,4 @@ package spark.scheduler.cluster +private[spark] class SlaveResources(val slaveId: String, val hostname: String, val coresFree: Int) {} diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 0bd2d15479..0043dbeb10 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -5,7 +5,7 @@ import spark.deploy.client.{Client, ClientListener} import spark.deploy.{Command, JobDescription} import scala.collection.mutable.HashMap -class SparkDeploySchedulerBackend( +private[spark] class SparkDeploySchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, @@ -16,6 +16,7 @@ class SparkDeploySchedulerBackend( var client: Client = null var stopping = false + var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt @@ -61,6 +62,9 @@ class SparkDeploySchedulerBackend( stopping = true; super.stop() client.stop() + if (shutdownCallback != null) { + shutdownCallback(this) + } } def connected(jobId: String) { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 80e8733671..1386cd9d44 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -4,19 +4,27 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer import spark.util.SerializableBuffer -sealed trait StandaloneClusterMessage extends Serializable +private[spark] sealed trait StandaloneClusterMessage extends Serializable // Master to slaves +private[spark] case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage + +private[spark] case class RegisteredSlave(sparkProperties: Seq[(String, String)]) extends StandaloneClusterMessage + +private[spark] case class RegisterSlaveFailed(message: String) extends StandaloneClusterMessage // Slaves to master +private[spark] case class RegisterSlave(slaveId: String, host: String, cores: Int) extends StandaloneClusterMessage +private[spark] case class StatusUpdate(slaveId: String, taskId: Long, state: TaskState, data: SerializableBuffer) extends StandaloneClusterMessage +private[spark] object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ def apply(slaveId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { @@ -25,5 +33,5 @@ object StatusUpdate { } // Internal messages in master -case object ReviveOffers extends StandaloneClusterMessage -case object StopMaster extends StandaloneClusterMessage +private[spark] case object ReviveOffers extends StandaloneClusterMessage +private[spark] case object StopMaster extends StandaloneClusterMessage diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 83e7c6e036..d2cce0dc05 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -16,6 +16,7 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient * Akka. These may be executed in a variety of ways, such as Mesos tasks for the coarse-grained * Mesos mode or standalone processes for Spark's standalone deploy mode (spark.deploy.*). */ +private[spark] class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) extends SchedulerBackend with Logging { @@ -149,6 +150,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) } -object StandaloneSchedulerBackend { +private[spark] object StandaloneSchedulerBackend { val ACTOR_NAME = "StandaloneScheduler" } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index f9a1b74fa5..aa097fd3a2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -3,7 +3,7 @@ package spark.scheduler.cluster import java.nio.ByteBuffer import spark.util.SerializableBuffer -class TaskDescription( +private[spark] class TaskDescription( val taskId: Long, val slaveId: String, val name: String, diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 65e59841a9..ca84503780 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -3,6 +3,7 @@ package spark.scheduler.cluster /** * Information about a running task attempt inside a TaskSet. */ +private[spark] class TaskInfo(val taskId: Long, val index: Int, val launchTime: Long, val host: String) { var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 5a7df6040c..9bb88ad6a1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,7 +17,7 @@ import java.nio.ByteBuffer /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -class TaskSetManager( +private[spark] class TaskSetManager( sched: ClusterScheduler, val taskSet: TaskSet) extends Logging { @@ -214,7 +214,8 @@ class TaskSetManager( } // Serialize and return the task val startTime = System.currentTimeMillis - val serializedTask = ser.serialize(task) + val serializedTask = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) @@ -243,6 +244,11 @@ class TaskSetManager( def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } val index = info.index info.markSuccessful() if (!finished(index)) { @@ -342,6 +348,7 @@ class TaskSetManager( failed = true causeOfFailure = message // TODO: Kill running tasks if we were not terminated due to a Mesos error + sched.listener.taskSetFailed(taskSet, message) sched.taskSetFinished(this) } diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index 1e83f103e7..6b919d68b2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -3,5 +3,6 @@ package spark.scheduler.cluster /** * Represents free resources available on a worker node. */ +private[spark] class WorkerOffer(val slaveId: String, val hostname: String, val cores: Int) { } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index eb47988f0c..2b38d8b52e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,9 +1,13 @@ package spark.scheduler.local +import java.io.File +import java.net.URLClassLoader import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable.HashMap import spark._ +import executor.ExecutorURLClassLoader import spark.scheduler._ /** @@ -11,15 +15,25 @@ import spark.scheduler._ * the scheduler also allows each task to fail up to maxFailures times, which is useful for * testing fault recovery. */ -class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with Logging { +private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkContext) + extends TaskScheduler + with Logging { + var attemptId = new AtomicInteger(0) var threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory) val env = SparkEnv.get var listener: TaskSchedulerListener = null + // Application dependencies (added through SparkContext) that we've fetched so far on this node. + // Each map holds the master's timestamp for the version of that file or JAR we got. + val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() + val currentJars: HashMap[String, Long] = new HashMap[String, Long]() + + val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader) + // TODO: Need to take into account stage priority in scheduling - override def start() {} + override def start() { } override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -43,15 +57,22 @@ class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { + Accumulators.clear() + Thread.currentThread().setContextClassLoader(classLoader) + // Serialize and deserialize the task so that accumulators are changed to thread-local ones; // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. - Accumulators.clear val ser = SparkEnv.get.closureSerializer.newInstance() - val bytes = ser.serialize(task) + val bytes = Task.serializeWithDependencies(task, sc.addedFiles, sc.addedJars, ser) logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes") + val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) + updateDependencies(taskFiles, taskJars) // Download any files added with addFile val deserializedTask = ser.deserialize[Task[_]]( - bytes, Thread.currentThread.getContextClassLoader) + taskBytes, Thread.currentThread.getContextClassLoader) + + // Run it val result: Any = deserializedTask.run(attemptId) + // Serialize and deserialize the result to emulate what the Mesos // executor does. This is useful to catch serialization errors early // on in development (so when users move their local Spark programs @@ -80,6 +101,31 @@ class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with submitTask(task, i) } } + + /** + * Download any missing dependencies if we receive a new set of files and JARs from the + * SparkContext. Also adds any new JARs we fetched to the class loader. + */ + private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { + // Fetch missing dependencies + for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name) + Utils.fetchFile(name, new File(".")) + currentFiles(name) = timestamp + } + for ((name, timestamp) <- newJars if currentFiles.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name) + Utils.fetchFile(name, new File(".")) + currentJars(name) = timestamp + // Add it to our class loader + val localName = name.split("/").last + val url = new File(".", localName).toURI.toURL + if (!classLoader.getURLs.contains(url)) { + logInfo("Adding " + url + " to class loader") + classLoader.addURL(url) + } + } + } override def stop() { threadPool.shutdownNow() diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 31784985dc..9737c6b63e 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -24,7 +24,7 @@ import spark.TaskState * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to * remove this. */ -class CoarseMesosSchedulerBackend( +private[spark] class CoarseMesosSchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, @@ -80,6 +80,8 @@ class CoarseMesosSchedulerBackend( "property, the SPARK_HOME environment variable or the SparkContext constructor") } + val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt + var nextMesosTaskId = 0 def newMesosTaskId(): Int = { @@ -177,7 +179,7 @@ class CoarseMesosSchedulerBackend( val task = MesosTaskInfo.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) .setSlaveId(offer.getSlaveId) - .setCommand(createCommand(offer, cpusToUse)) + .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", executorMemory)) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 44eda93dd1..e85e4ef318 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -20,7 +20,7 @@ import spark.TaskState * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks * from multiple apps can run on different cores) and in time (a core can switch ownership). */ -class MesosSchedulerBackend( +private[spark] class MesosSchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 45f99717bc..21a2901548 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1,33 +1,25 @@ package spark.storage -import java.io._ -import java.nio._ -import java.nio.channels.FileChannel.MapMode -import java.util.{HashMap => JHashMap} -import java.util.LinkedHashMap -import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.LinkedBlockingQueue -import java.util.Collections - import akka.dispatch.{Await, Future} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ +import akka.util.Duration -import it.unimi.dsi.fastutil.io._ +import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.CacheTracker -import spark.Logging -import spark.Serializer -import spark.SizeEstimator -import spark.SparkEnv -import spark.SparkException -import spark.Utils -import spark.util.ByteBufferInputStream +import java.io.{InputStream, OutputStream, Externalizable, ObjectInput, ObjectOutput} +import java.nio.{MappedByteBuffer, ByteBuffer} +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.JavaConversions._ + +import spark.{CacheTracker, Logging, Serializer, SizeEstimator, SparkException, Utils} import spark.network._ -import akka.util.Duration +import spark.util.ByteBufferInputStream +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} +import sun.nio.ch.DirectBuffer -class BlockManagerId(var ip: String, var port: Int) extends Externalizable { + +private[spark] class BlockManagerId(var ip: String, var port: Int) extends Externalizable { def this() = this(null, 0) override def writeExternal(out: ObjectOutput) { @@ -51,41 +43,61 @@ class BlockManagerId(var ip: String, var port: Int) extends Externalizable { } -case class BlockException(blockId: String, message: String, ex: Exception = null) extends Exception(message) +private[spark] +case class BlockException(blockId: String, message: String, ex: Exception = null) +extends Exception(message) -class BlockLocker(numLockers: Int) { +private[spark] class BlockLocker(numLockers: Int) { private val hashLocker = Array.fill(numLockers)(new Object()) - + def getLock(blockId: String): Object = { return hashLocker(math.abs(blockId.hashCode % numLockers)) } } +private[spark] class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, maxMemory: Long) extends Logging { - case class BlockInfo(level: StorageLevel, tellMaster: Boolean) + class BlockInfo(val level: StorageLevel, val tellMaster: Boolean, var pending: Boolean = true) { + def waitForReady() { + if (pending) { + synchronized { + while (pending) this.wait() + } + } + } + + def markReady() { + pending = false + synchronized { + this.notifyAll() + } + } + } private val NUM_LOCKS = 337 private val locker = new BlockLocker(NUM_LOCKS) private val blockInfo = new ConcurrentHashMap[String, BlockInfo]() - private val memoryStore: BlockStore = new MemoryStore(this, maxMemory) - private val diskStore: BlockStore = new DiskStore(this, - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) - + + private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) + private[storage] val diskStore: BlockStore = + new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext - + val connectionManagerId = connectionManager.id val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port) - + // TODO: This will be removed after cacheTracker is removed from the code base. var cacheTracker: CacheTracker = null - initLogging() + val numParallelFetches = BlockManager.getNumParallelFetchesFromSystemProperties + val compress = System.getProperty("spark.blockManager.compress", "false").toBoolean initialize() @@ -102,7 +114,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m */ private def initialize() { master.mustRegisterBlockManager( - RegisterBlockManager(blockManagerId, maxMemory, maxMemory)) + RegisterBlockManager(blockManagerId, maxMemory)) BlockManagerWorker.startBlockManagerWorker(this) } @@ -115,36 +127,32 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } /** - * Change storage level for a local block and tell master is necesary. - * If new level is invalid, then block info (if it exists) will be silently removed. + * Tell the master about the current storage status of a block. This will send a heartbeat + * 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. */ - def setLevel(blockId: String, level: StorageLevel, tellMaster: Boolean = true) { - if (level == null) { - throw new IllegalArgumentException("Storage level is null") - } - - // If there was earlier info about the block, then use earlier tellMaster - val oldInfo = blockInfo.get(blockId) - val newTellMaster = if (oldInfo != null) oldInfo.tellMaster else tellMaster - if (oldInfo != null && oldInfo.tellMaster != tellMaster) { - logWarning("Ignoring tellMaster setting as it is different from earlier setting") - } - - // If level is valid, store the block info, else remove the block info - if (level.isValid) { - blockInfo.put(blockId, new BlockInfo(level, newTellMaster)) - logDebug("Info for block " + blockId + " updated with new level as " + level) - } else { - blockInfo.remove(blockId) - logDebug("Info for block " + blockId + " removed as new level is null or invalid") - } - - // Tell master if necessary - if (newTellMaster) { + def reportBlockStatus(blockId: String) { + locker.getLock(blockId).synchronized { + val curLevel = blockInfo.get(blockId) match { + case null => + StorageLevel.NONE + case info => + info.level match { + case null => + StorageLevel.NONE + case level => + val inMem = level.useMemory && memoryStore.contains(blockId) + val onDisk = level.useDisk && diskStore.contains(blockId) + new StorageLevel(onDisk, inMem, level.deserialized, level.replication) + } + } + master.mustHeartBeat(HeartBeat( + blockManagerId, + blockId, + curLevel, + if (curLevel.useMemory) memoryStore.getSize(blockId) else 0L, + if (curLevel.useDisk) diskStore.getSize(blockId) else 0L)) logDebug("Told master about block " + blockId) - notifyMaster(HeartBeat(blockManagerId, blockId, level, 0, 0)) - } else { - logDebug("Did not tell master about block " + blockId) } } @@ -174,55 +182,122 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m * Get block from local block manager. */ def getLocal(blockId: String): Option[Iterator[Any]] = { - if (blockId == null) { - throw new IllegalArgumentException("Block Id is null") - } logDebug("Getting local block " + blockId) locker.getLock(blockId).synchronized { - - // Check storage level of block - val level = getLevel(blockId) - if (level != null) { - logDebug("Level for block " + blockId + " is " + level + " on local machine") - + val info = blockInfo.get(blockId) + if (info != null) { + info.waitForReady() // In case the block is still being put() by another thread + val level = info.level + logDebug("Level for block " + blockId + " is " + level) + // Look for the block in memory if (level.useMemory) { logDebug("Getting block " + blockId + " from memory") memoryStore.getValues(blockId) match { - case Some(iterator) => { - logDebug("Block " + blockId + " found in memory") + case Some(iterator) => return Some(iterator) - } - case None => { + case None => logDebug("Block " + blockId + " not found in memory") - } } - } else { - logDebug("Not getting block " + blockId + " from memory") } - // Look for block in disk + // Look for block on disk, potentially loading it back into memory if required if (level.useDisk) { logDebug("Getting block " + blockId + " from disk") - diskStore.getValues(blockId) match { - case Some(iterator) => { - logDebug("Block " + blockId + " found in disk") - return Some(iterator) + if (level.useMemory && level.deserialized) { + diskStore.getValues(blockId) match { + case Some(iterator) => + // Put the block back in memory before returning it + memoryStore.putValues(blockId, iterator, level, true) match { + case Left(iterator2) => + return Some(iterator2) + case _ => + throw new Exception("Memory store did not return back an iterator") + } + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") + } + } else if (level.useMemory && !level.deserialized) { + // Read it as a byte buffer into memory first, then return it + diskStore.getBytes(blockId) match { + case Some(bytes) => + // Put a copy of the block back in memory before returning it. Note that we can't + // put the ByteBuffer returned by the disk store as that's a memory-mapped file. + val copyForMemory = ByteBuffer.allocate(bytes.limit) + copyForMemory.put(bytes) + memoryStore.putBytes(blockId, copyForMemory, level) + bytes.rewind() + return Some(dataDeserialize(bytes)) + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") } - case None => { - throw new Exception("Block " + blockId + " not found in disk") - return None + } else { + diskStore.getValues(blockId) match { + case Some(iterator) => + return Some(iterator) + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") } } - } else { - logDebug("Not getting block " + blockId + " from disk") } + } else { + logDebug("Block " + blockId + " not registered locally") + } + } + return None + } + /** + * Get block from the local block manager as serialized bytes. + */ + def getLocalBytes(blockId: String): Option[ByteBuffer] = { + // TODO: This whole thing is very similar to getLocal; we need to refactor it somehow + logDebug("Getting local block " + blockId + " as bytes") + locker.getLock(blockId).synchronized { + val info = blockInfo.get(blockId) + if (info != null) { + info.waitForReady() // In case the block is still being put() by another thread + val level = info.level + logDebug("Level for block " + blockId + " is " + level) + + // Look for the block in memory + if (level.useMemory) { + logDebug("Getting block " + blockId + " from memory") + memoryStore.getBytes(blockId) match { + case Some(bytes) => + return Some(bytes) + case None => + logDebug("Block " + blockId + " not found in memory") + } + } + + // Look for block on disk + if (level.useDisk) { + // Read it as a byte buffer into memory first, then return it + diskStore.getBytes(blockId) match { + case Some(bytes) => + if (level.useMemory) { + if (level.deserialized) { + memoryStore.putBytes(blockId, bytes, level) + } else { + // The memory store will hang onto the ByteBuffer, so give it a copy instead of + // the memory-mapped file buffer we got from the disk store + val copyForMemory = ByteBuffer.allocate(bytes.limit) + copyForMemory.put(bytes) + memoryStore.putBytes(blockId, copyForMemory, level) + } + } + bytes.rewind() + return Some(bytes) + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") + } + } } else { - logDebug("Level for block " + blockId + " not found") + logDebug("Block " + blockId + " not registered locally") } - } - return None + } + return None } /** @@ -272,70 +347,93 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m val totalBlocks = blocksByAddress.map(_._2.size).sum logDebug("Getting " + totalBlocks + " blocks") var startTime = System.currentTimeMillis - val results = new LinkedBlockingQueue[(String, Option[Iterator[Any]])] val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new ArrayBuffer[String]() - val remoteBlockIdsPerLocation = new HashMap[BlockManagerId, Seq[String]]() + val remoteBlockIds = new HashSet[String]() - // Split local and remote blocks - for ((address, blockIds) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockIds - } else { - remoteBlockIds ++= blockIds - remoteBlockIdsPerLocation(address) = blockIds - } - } - - // Start getting remote blocks - for ((bmId, bIds) <- remoteBlockIdsPerLocation) { - val cmId = ConnectionManagerId(bmId.ip, bmId.port) - val blockMessages = bIds.map(bId => BlockMessage.fromGetBlock(GetBlock(bId))) + // A queue to hold our results. Because we want all the deserializing the happen in the + // caller's thread, this will actually hold functions to produce the Iterator for each block. + // For local blocks we'll have an iterator already, while for remote ones we'll deserialize. + val results = new LinkedBlockingQueue[(String, Option[() => Iterator[Any]])] + + // Bound the number and memory usage of fetched remote blocks. + val blocksToRequest = new Queue[(BlockManagerId, BlockMessage)] + + def sendRequest(bmId: BlockManagerId, blockMessages: Seq[BlockMessage]) { + val cmId = new ConnectionManagerId(bmId.ip, bmId.port) val blockMessageArray = new BlockMessageArray(blockMessages) val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - blockMessageArray.foreach(blockMessage => { + for (blockMessage <- blockMessageArray) { if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { throw new SparkException( "Unexpected message " + blockMessage.getType + " received from " + cmId) } - val buffer = blockMessage.getData val blockId = blockMessage.getId - val block = dataDeserialize(buffer) - results.put((blockId, Some(block))) + results.put((blockId, Some(() => dataDeserialize(blockMessage.getData)))) logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - }) + } } case None => { - logError("Could not get blocks from " + cmId) - for (blockId <- bIds) { - results.put((blockId, None)) + logError("Could not get block(s) from " + cmId) + for (blockMessage <- blockMessages) { + results.put((blockMessage.getId, None)) } } } } - logDebug("Started remote gets for " + remoteBlockIds.size + " blocks in " + + + // Split local and remote blocks. Remote blocks are further split into ones that will + // be requested initially and ones that will be added to a queue of blocks to request. + val initialRequestBlocks = new HashMap[BlockManagerId, ArrayBuffer[BlockMessage]]() + var initialRequests = 0 + val blocksToGetLater = new ArrayBuffer[(BlockManagerId, BlockMessage)] + for ((address, blockIds) <- Utils.randomize(blocksByAddress)) { + if (address == blockManagerId) { + localBlockIds ++= blockIds + } else { + remoteBlockIds ++= blockIds + for (blockId <- blockIds) { + val blockMessage = BlockMessage.fromGetBlock(GetBlock(blockId)) + if (initialRequests < numParallelFetches) { + initialRequestBlocks.getOrElseUpdate(address, new ArrayBuffer[BlockMessage]) + .append(blockMessage) + initialRequests += 1 + } else { + blocksToGetLater.append((address, blockMessage)) + } + } + } + } + // Add the remaining blocks into a queue to pull later in a random order + blocksToRequest ++= Utils.randomize(blocksToGetLater) + + // Send out initial request(s) for 'numParallelFetches' blocks. + for ((bmId, blockMessages) <- initialRequestBlocks) { + sendRequest(bmId, blockMessages) + } + + logDebug("Started remote gets for " + numParallelFetches + " blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - // Get the local blocks while remote blocks are being fetched + // Get the local blocks while remote blocks are being fetched. startTime = System.currentTimeMillis - localBlockIds.foreach(id => { - get(id) match { + for (id <- localBlockIds) { + getLocal(id) match { case Some(block) => { - results.put((id, Some(block))) + results.put((id, Some(() => block))) logDebug("Got local block " + id) } case None => { throw new BlockException(id, "Could not get block " + id + " from local machine") } } - }) + } logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - // Return an iterator that will read fetched blocks off the queue as they arrive + // Return an iterator that will read fetched blocks off the queue as they arrive. return new Iterator[(String, Option[Iterator[Any]])] { var resultsGotten = 0 @@ -343,7 +441,12 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 - results.take() + val (blockId, functionOption) = results.take() + if (remoteBlockIds.contains(blockId) && !blocksToRequest.isEmpty) { + val (bmId, blockMessage) = blocksToRequest.dequeue() + sendRequest(bmId, Seq(blockMessage)) + } + (blockId, functionOption.map(_.apply())) } } } @@ -362,7 +465,18 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m throw new IllegalArgumentException("Storage level is null or invalid") } - val startTimeMs = System.currentTimeMillis + if (blockInfo.containsKey(blockId)) { + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return + } + + // Remember the block's storage level so that we can correctly drop it to disk if it needs + // to be dropped right after it got put into memory. Note, however, that other threads will + // not be able to get() this block until we call markReady on its BlockInfo. + val myInfo = new BlockInfo(level, tellMaster) + blockInfo.put(blockId, myInfo) + + val startTimeMs = System.currentTimeMillis var bytes: ByteBuffer = null // If we need to replicate the data, we'll want access to the values, but because our @@ -370,49 +484,37 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m // the put serializes data, we'll remember the bytes, above; but for the case where // it doesn't, such as MEMORY_ONLY_DESER, let's rely on the put returning an Iterator. var valuesAfterPut: Iterator[Any] = null - + locker.getLock(blockId).synchronized { logDebug("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") - - // Check and warn if block with same id already exists - if (getLevel(blockId) != null) { - logWarning("Block " + blockId + " already exists in local machine") - return - } - if (level.useMemory && level.useDisk) { - // If saving to both memory and disk, then serialize only once - memoryStore.putValues(blockId, values, level) match { - case Left(newValues) => - diskStore.putValues(blockId, newValues, level) match { - case Right(newBytes) => bytes = newBytes - case _ => throw new Exception("Unexpected return value") - } - case Right(newBytes) => - bytes = newBytes - diskStore.putBytes(blockId, newBytes, level) - } - } else if (level.useMemory) { - // If only save to memory - memoryStore.putValues(blockId, values, level) match { + if (level.useMemory) { + // Save it just to memory first, even if it also has useDisk set to true; we will later + // drop it to disk if the memory store can't hold it. + memoryStore.putValues(blockId, values, level, true) match { case Right(newBytes) => bytes = newBytes case Left(newIterator) => valuesAfterPut = newIterator } } else { - // If only save to disk - diskStore.putValues(blockId, values, level) match { + // Save directly to disk. + val askForBytes = level.replication > 1 // Don't get back the bytes unless we replicate them + diskStore.putValues(blockId, values, level, askForBytes) match { case Right(newBytes) => bytes = newBytes - case _ => throw new Exception("Unexpected return value") + case _ => } } - - // Store the storage level - setLevel(blockId, level, tellMaster) + + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + myInfo.markReady() + if (tellMaster) { + reportBlockStatus(blockId) + } } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) - // Replicate block if required + // Replicate block if required if (level.replication > 1) { // Serialize the block if not already done if (bytes == null) { @@ -422,9 +524,11 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } bytes = dataSerialize(valuesAfterPut) } - replicate(blockId, bytes, level) + replicate(blockId, bytes, level) } + BlockManager.dispose(bytes) + // TODO: This code will be removed when CacheTracker is gone. if (blockId.startsWith("rdd")) { notifyTheCacheTracker(blockId) @@ -436,7 +540,9 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m /** * Put a new block of serialized bytes to the block manager. */ - def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) { + def putBytes( + blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) { + if (blockId == null) { throw new IllegalArgumentException("Block Id is null") } @@ -446,14 +552,26 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m if (level == null || !level.isValid) { throw new IllegalArgumentException("Storage level is null or invalid") } - - val startTimeMs = System.currentTimeMillis - - // Initiate the replication before storing it locally. This is faster as + + if (blockInfo.containsKey(blockId)) { + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return + } + + // Remember the block's storage level so that we can correctly drop it to disk if it needs + // to be dropped right after it got put into memory. Note, however, that other threads will + // not be able to get() this block until we call markReady on its BlockInfo. + val myInfo = new BlockInfo(level, tellMaster) + blockInfo.put(blockId, myInfo) + + val startTimeMs = System.currentTimeMillis + + // Initiate the replication before storing it locally. This is faster as // data is already serialized and ready for sending val replicationFuture = if (level.replication > 1) { + val bufferView = bytes.duplicate() // Doesn't copy the bytes, just creates a wrapper Future { - replicate(blockId, bytes, level) + replicate(blockId, bufferView, level) } } else { null @@ -462,27 +580,29 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m locker.getLock(blockId).synchronized { logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") - if (getLevel(blockId) != null) { - logWarning("Block " + blockId + " already exists") - return - } if (level.useMemory) { + // Store it only in memory at first, even if useDisk is also set to true + bytes.rewind() memoryStore.putBytes(blockId, bytes, level) - } - if (level.useDisk) { + } else { + bytes.rewind() diskStore.putBytes(blockId, bytes, level) } - // Store the storage level - setLevel(blockId, level, tellMaster) + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + myInfo.markReady() + if (tellMaster) { + reportBlockStatus(blockId) + } } // TODO: This code will be removed when CacheTracker is gone. if (blockId.startsWith("rdd")) { notifyTheCacheTracker(blockId) } - + // If replication had started, then wait for it to finish if (level.replication > 1) { if (replicationFuture == null) { @@ -491,12 +611,11 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m Await.ready(replicationFuture, Duration.Inf) } - val finishTime = System.currentTimeMillis if (level.replication > 1) { - logDebug("PutBytes for block " + blockId + " with replication took " + + logDebug("PutBytes for block " + blockId + " with replication took " + Utils.getUsedTimeMs(startTimeMs)) } else { - logDebug("PutBytes for block " + blockId + " without replication took " + + logDebug("PutBytes for block " + blockId + " without replication took " + Utils.getUsedTimeMs(startTimeMs)) } } @@ -504,39 +623,44 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m /** * Replicate block to another node. */ - + var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) { val tLevel: StorageLevel = new StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) - var peers = master.mustGetPeers(GetPeers(blockManagerId, level.replication - 1)) - for (peer: BlockManagerId <- peers) { + if (cachedPeers == null) { + cachedPeers = master.mustGetPeers(GetPeers(blockManagerId, level.replication - 1)) + } + for (peer: BlockManagerId <- cachedPeers) { val start = System.nanoTime + data.rewind() logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is " - + data.array().length + " Bytes. To node: " + peer) + + data.limit() + " Bytes. To node: " + peer) if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel), new ConnectionManagerId(peer.ip, peer.port))) { logError("Failed to call syncPutBlock to " + peer) } logDebug("Replicated BlockId " + blockId + " once used " + (System.nanoTime - start) / 1e6 + " s; The size of the data is " + - data.array().length + " bytes.") + data.limit() + " bytes.") } } // TODO: This code will be removed when CacheTracker is gone. private def notifyTheCacheTracker(key: String) { - val rddInfo = key.split(":") - val rddId: Int = rddInfo(1).toInt - val splitIndex: Int = rddInfo(2).toInt - val host = System.getProperty("spark.hostname", Utils.localHostName) - cacheTracker.notifyTheCacheTrackerFromBlockManager(spark.AddedToCache(rddId, splitIndex, host)) + if (cacheTracker != null) { + val rddInfo = key.split("_") + val rddId: Int = rddInfo(1).toInt + val partition: Int = rddInfo(2).toInt + val host = System.getProperty("spark.hostname", Utils.localHostName()) + cacheTracker.notifyTheCacheTrackerFromBlockManager(spark.AddedToCache(rddId, partition, host)) + } } /** * Read a block consisting of a single object. */ def getSingle(blockId: String): Option[Any] = { - get(blockId).map(_.next) + get(blockId).map(_.next()) } /** @@ -547,42 +671,64 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } /** - * Drop block from memory (called when memory store has reached it limit) + * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory + * store reaches its limit and needs to free up space. */ - def dropFromMemory(blockId: String) { + def dropFromMemory(blockId: String, data: Either[Iterator[_], ByteBuffer]) { + logInfo("Dropping block " + blockId + " from memory") locker.getLock(blockId).synchronized { - val level = getLevel(blockId) - if (level == null) { - logWarning("Block " + blockId + " cannot be removed from memory as it does not exist") - return + val info = blockInfo.get(blockId) + val level = info.level + if (level.useDisk && !diskStore.contains(blockId)) { + logInfo("Writing block " + blockId + " to disk") + data match { + case Left(iterator) => + diskStore.putValues(blockId, iterator, level, false) + case Right(bytes) => + diskStore.putBytes(blockId, bytes, level) + } } - if (!level.useMemory) { - logWarning("Block " + blockId + " cannot be removed from memory as it is not in memory") - return + memoryStore.remove(blockId) + if (info.tellMaster) { + reportBlockStatus(blockId) + } + if (!level.useDisk) { + // The block is completely gone from this node; forget it so we can put() it again later. + blockInfo.remove(blockId) } - memoryStore.remove(blockId) - val newLevel = new StorageLevel(level.useDisk, false, level.deserialized, level.replication) - setLevel(blockId, newLevel) } } + /** + * Wrap an output stream for compression if block compression is enabled + */ + def wrapForCompression(s: OutputStream): OutputStream = { + if (compress) new LZFOutputStream(s) else s + } + + /** + * Wrap an input stream for compression if block compression is enabled + */ + def wrapForCompression(s: InputStream): InputStream = { + if (compress) new LZFInputStream(s) else s + } + def dataSerialize(values: Iterator[Any]): ByteBuffer = { - /*serializer.newInstance().serializeMany(values)*/ val byteStream = new FastByteArrayOutputStream(4096) - serializer.newInstance().serializeStream(byteStream).writeAll(values).close() + val ser = serializer.newInstance() + ser.serializeStream(wrapForCompression(byteStream)).writeAll(values).close() byteStream.trim() ByteBuffer.wrap(byteStream.array) } + /** + * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of + * the iterator is reached. + */ def dataDeserialize(bytes: ByteBuffer): Iterator[Any] = { - /*serializer.newInstance().deserializeMany(bytes)*/ - val ser = serializer.newInstance() bytes.rewind() - return ser.deserializeStream(new ByteBufferInputStream(bytes)).toIterator - } - - private def notifyMaster(heartBeat: HeartBeat) { - master.mustHeartBeat(heartBeat) + val ser = serializer.newInstance() + ser.deserializeStream(wrapForCompression(new ByteBufferInputStream(bytes, true))).asIterator } def stop() { @@ -594,9 +740,29 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } } -object BlockManager { - def getMaxMemoryFromSystemProperties(): Long = { +private[spark] +object BlockManager extends Logging { + def getNumParallelFetchesFromSystemProperties: Int = { + System.getProperty("spark.blockManager.parallelFetches", "4").toInt + } + + def getMaxMemoryFromSystemProperties: Long = { val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } + + /** + * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that + * might cause errors if one attempts to read from the unmapped buffer, but it's better than + * waiting for the GC to find it because that could lead to huge numbers of open files. There's + * unfortunately no standard API to do this. + */ + def dispose(buffer: ByteBuffer) { + if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { + logDebug("Unmapping " + buffer) + if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) { + buffer.asInstanceOf[DirectBuffer].cleaner().clean() + } + } + } } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 9f03c5a32c..7bfa31ac3d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -3,37 +3,35 @@ package spark.storage import java.io._ import java.util.{HashMap => JHashMap} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import akka.actor._ import akka.dispatch._ import akka.pattern.ask import akka.remote._ -import akka.util.Duration -import akka.util.Timeout +import akka.util.{Duration, Timeout} import akka.util.duration._ -import spark.Logging -import spark.SparkException -import spark.Utils +import spark.{Logging, SparkException, Utils} + +private[spark] sealed trait ToBlockManagerMaster +private[spark] case class RegisterBlockManager( blockManagerId: BlockManagerId, - maxMemSize: Long, - maxDiskSize: Long) + maxMemSize: Long) extends ToBlockManagerMaster - + +private[spark] class HeartBeat( var blockManagerId: BlockManagerId, var blockId: String, var storageLevel: StorageLevel, - var deserializedSize: Long, - var size: Long) + var memSize: Long, + var diskSize: Long) extends ToBlockManagerMaster with Externalizable { @@ -43,8 +41,8 @@ class HeartBeat( blockManagerId.writeExternal(out) out.writeUTF(blockId) storageLevel.writeExternal(out) - out.writeInt(deserializedSize.toInt) - out.writeInt(size.toInt) + out.writeInt(memSize.toInt) + out.writeInt(diskSize.toInt) } override def readExternal(in: ObjectInput) { @@ -53,84 +51,101 @@ class HeartBeat( blockId = in.readUTF() storageLevel = new StorageLevel() storageLevel.readExternal(in) - deserializedSize = in.readInt() - size = in.readInt() + memSize = in.readInt() + diskSize = in.readInt() } } +private[spark] object HeartBeat { def apply(blockManagerId: BlockManagerId, blockId: String, storageLevel: StorageLevel, - deserializedSize: Long, - size: Long): HeartBeat = { - new HeartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size) + memSize: Long, + diskSize: Long): HeartBeat = { + new HeartBeat(blockManagerId, blockId, storageLevel, memSize, diskSize) } - // For pattern-matching def unapply(h: HeartBeat): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.deserializedSize, h.size)) + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) } } +private[spark] case class GetLocations(blockId: String) extends ToBlockManagerMaster +private[spark] case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster +private[spark] case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster +private[spark] case class RemoveHost(host: String) extends ToBlockManagerMaster +private[spark] case object StopBlockManagerMaster extends ToBlockManagerMaster -class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { +private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { class BlockManagerInfo( + val blockManagerId: BlockManagerId, timeMs: Long, - maxMem: Long, - maxDisk: Long) { + val maxMem: Long) { private var lastSeenMs = timeMs - private var remainedMem = maxMem - private var remainedDisk = maxDisk + private var remainingMem = maxMem private val blocks = new JHashMap[String, StorageLevel] + + logInfo("Registering block manager %s:%d with %s RAM".format( + blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem))) def updateLastSeenMs() { lastSeenMs = System.currentTimeMillis() / 1000 } - def addBlock(blockId: String, storageLevel: StorageLevel, deserializedSize: Long, size: Long) = - synchronized { + def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long) + : Unit = synchronized { + updateLastSeenMs() if (blocks.containsKey(blockId)) { - val oriLevel: StorageLevel = blocks.get(blockId) + // The block exists on the slave already. + val originalLevel: StorageLevel = blocks.get(blockId) - if (oriLevel.deserialized) { - remainedMem += deserializedSize - } - if (oriLevel.useMemory) { - remainedMem += size - } - if (oriLevel.useDisk) { - remainedDisk += size + if (originalLevel.useMemory) { + remainingMem += memSize } } - if (storageLevel.isValid) { + if (storageLevel.isValid) { + // isValid means it is either stored in-memory or on-disk. blocks.put(blockId, storageLevel) - if (storageLevel.deserialized) { - remainedMem -= deserializedSize - } if (storageLevel.useMemory) { - remainedMem -= size + remainingMem -= memSize + logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + Utils.memoryBytesToString(remainingMem))) } if (storageLevel.useDisk) { - remainedDisk -= size + logInfo("Added %s on disk on %s:%d (size: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) } - } else { + } else if (blocks.containsKey(blockId)) { + // If isValid is not true, drop the block. + val originalLevel: StorageLevel = blocks.get(blockId) blocks.remove(blockId) + if (originalLevel.useMemory) { + remainingMem += memSize + logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + Utils.memoryBytesToString(remainingMem))) + } + if (originalLevel.useDisk) { + logInfo("Removed %s on %s:%d on disk (size: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) + } } } @@ -139,15 +154,11 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } def getRemainedMem: Long = { - return remainedMem - } - - def getRemainedDisk: Long = { - return remainedDisk + return remainingMem } override def toString: String = { - return "BlockManagerInfo " + timeMs + " " + remainedMem + " " + remainedDisk + return "BlockManagerInfo " + timeMs + " " + remainingMem } def clear() { @@ -171,8 +182,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } def receive = { - case RegisterBlockManager(blockManagerId, maxMemSize, maxDiskSize) => - register(blockManagerId, maxMemSize, maxDiskSize) + case RegisterBlockManager(blockManagerId, maxMemSize) => + register(blockManagerId, maxMemSize) case HeartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size) => heartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size) @@ -200,16 +211,15 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { logInfo("Got unknown message: " + other) } - private def register(blockManagerId: BlockManagerId, maxMemSize: Long, maxDiskSize: Long) { + private def register(blockManagerId: BlockManagerId, maxMemSize: Long) { val startTimeMs = System.currentTimeMillis() val tmp = " " + blockManagerId + " " logDebug("Got in register 0" + tmp + Utils.getUsedTimeMs(startTimeMs)) - logInfo("Got Register Msg from " + blockManagerId) if (blockManagerId.ip == Utils.localHostName() && !isLocal) { logInfo("Got Register Msg from master node, don't register it") } else { blockManagerInfo += (blockManagerId -> new BlockManagerInfo( - System.currentTimeMillis() / 1000, maxMemSize, maxDiskSize)) + blockManagerId, System.currentTimeMillis() / 1000, maxMemSize)) } logDebug("Got in register 1" + tmp + Utils.getUsedTimeMs(startTimeMs)) sender ! true @@ -219,8 +229,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { blockManagerId: BlockManagerId, blockId: String, storageLevel: StorageLevel, - deserializedSize: Long, - size: Long) { + memSize: Long, + diskSize: Long) { val startTimeMs = System.currentTimeMillis() val tmp = " " + blockManagerId + " " + blockId + " " @@ -231,7 +241,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { sender ! true } - blockManagerInfo(blockManagerId).addBlock(blockId, storageLevel, deserializedSize, size) + blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) var locations: HashSet[BlockManagerId] = null if (blockInfo.containsKey(blockId)) { @@ -329,7 +339,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } } -class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: Boolean) +private[spark] class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: Boolean) extends Logging { val AKKA_ACTOR_NAME: String = "BlockMasterManager" @@ -386,10 +396,12 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B } def mustRegisterBlockManager(msg: RegisterBlockManager) { + logInfo("Trying to register BlockManager") while (! syncRegisterBlockManager(msg)) { logWarning("Failed to register " + msg) Thread.sleep(REQUEST_RETRY_INTERVAL_MS) } + logInfo("Done registering BlockManager") } def syncRegisterBlockManager(msg: RegisterBlockManager): Boolean = { diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index d74cdb38a8..f72079e267 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -1,25 +1,23 @@ package spark.storage -import java.nio._ +import java.nio.ByteBuffer import scala.actors._ import scala.actors.Actor._ import scala.actors.remote._ - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random -import spark.Logging -import spark.Utils -import spark.SparkEnv +import spark.{Logging, Utils, SparkEnv} import spark.network._ /** - * This should be changed to use event model late. + * A network interface for BlockManager. Each slave should have one + * BlockManagerWorker. + * + * TODO: Use event model. */ -class BlockManagerWorker(val blockManager: BlockManager) extends Logging { +private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { initLogging() blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) @@ -32,11 +30,11 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging { logDebug("Handling as a buffer message " + bufferMessage) val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) logDebug("Parsed as a block message array") - val responseMessages = blockMessages.map(processBlockMessage _).filter(_ != None).map(_.get) + val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) /*logDebug("Processed block messages")*/ return Some(new BlockMessageArray(responseMessages).toBufferMessage) } catch { - case e: Exception => logError("Exception handling buffer message: " + e.getMessage) + case e: Exception => logError("Exception handling buffer message", e) return None } } @@ -73,22 +71,15 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging { logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes) blockManager.putBytes(id, bytes, level) logDebug("PutBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " with data size: " + bytes.array().length) + + " with data size: " + bytes.limit) } private def getBlock(id: String): ByteBuffer = { val startTimeMs = System.currentTimeMillis() - logDebug("Getblock " + id + " started from " + startTimeMs) - val block = blockManager.getLocal(id) - val buffer = block match { - case Some(tValues) => { - val values = tValues - val buffer = blockManager.dataSerialize(values) - buffer - } - case None => { - null - } + logDebug("GetBlock " + id + " started from " + startTimeMs) + val buffer = blockManager.getLocalBytes(id) match { + case Some(bytes) => bytes + case None => null } logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) + " and got buffer " + buffer) @@ -96,7 +87,7 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging { } } -object BlockManagerWorker extends Logging { +private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null private val DATA_TRANSFER_TIME_OUT_MS: Long = 500 private val REQUEST_RETRY_INTERVAL_MS: Long = 1000 diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index 0b2ed69e07..3f234df654 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -1,6 +1,6 @@ package spark.storage -import java.nio._ +import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer @@ -8,11 +8,11 @@ import scala.collection.mutable.ArrayBuffer import spark._ import spark.network._ -case class GetBlock(id: String) -case class GotBlock(id: String, data: ByteBuffer) -case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel) +private[spark] case class GetBlock(id: String) +private[spark] case class GotBlock(id: String, data: ByteBuffer) +private[spark] case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel) -class BlockMessage() extends Logging{ +private[spark] class BlockMessage() { // Un-initialized: typ = 0 // GetBlock: typ = 1 // GotBlock: typ = 2 @@ -22,8 +22,6 @@ class BlockMessage() extends Logging{ private var data: ByteBuffer = null private var level: StorageLevel = null - initLogging() - def set(getBlock: GetBlock) { typ = BlockMessage.TYPE_GET_BLOCK id = getBlock.id @@ -62,8 +60,6 @@ class BlockMessage() extends Logging{ } id = idBuilder.toString() - logDebug("Set from buffer Result: " + typ + " " + id) - logDebug("Buffer position is " + buffer.position) if (typ == BlockMessage.TYPE_PUT_BLOCK) { val booleanInt = buffer.getInt() @@ -77,23 +73,18 @@ class BlockMessage() extends Logging{ } data.put(buffer) data.flip() - logDebug("Set from buffer Result 2: " + level + " " + data) } else if (typ == BlockMessage.TYPE_GOT_BLOCK) { val dataLength = buffer.getInt() - logDebug("Data length is "+ dataLength) - logDebug("Buffer position is " + buffer.position) data = ByteBuffer.allocate(dataLength) if (dataLength != buffer.remaining) { throw new Exception("Error parsing buffer") } data.put(buffer) data.flip() - logDebug("Set from buffer Result 3: " + data) } val finishTime = System.currentTimeMillis - logDebug("Converted " + id + " from bytebuffer in " + (finishTime - startTime) / 1000.0 + " s") } def set(bufferMsg: BufferMessage) { @@ -145,8 +136,6 @@ class BlockMessage() extends Logging{ buffers += data } - logDebug("Start to log buffers.") - buffers.foreach((x: ByteBuffer) => logDebug("" + x)) /* println() println("BlockMessage: ") @@ -160,7 +149,6 @@ class BlockMessage() extends Logging{ println() */ val finishTime = System.currentTimeMillis - logDebug("Converted " + id + " to buffer message in " + (finishTime - startTime) / 1000.0 + " s") return Message.createBufferMessage(buffers) } @@ -170,7 +158,7 @@ class BlockMessage() extends Logging{ } } -object BlockMessage { +private[spark] object BlockMessage { val TYPE_NON_INITIALIZED: Int = 0 val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 @@ -208,7 +196,7 @@ object BlockMessage { def main(args: Array[String]) { val B = new BlockMessage() - B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.DISK_AND_MEMORY_2)) + B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2)) val bMsg = B.toBufferMessage val C = new BlockMessage() C.set(bMsg) diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index 497a19856e..a25decb123 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -1,5 +1,6 @@ package spark.storage -import java.nio._ + +import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer @@ -7,6 +8,7 @@ import scala.collection.mutable.ArrayBuffer import spark._ import spark.network._ +private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { def this(bm: BlockMessage) = this(Array(bm)) @@ -84,7 +86,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM } } -object BlockMessageArray { +private[spark] object BlockMessageArray { def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() @@ -98,7 +100,7 @@ object BlockMessageArray { if (i % 2 == 0) { val buffer = ByteBuffer.allocate(100) buffer.clear - BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY)) + BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY_SER)) } else { BlockMessage.fromGetBlock(GetBlock(i.toString)) } diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala index 17f4f51aa8..ff482ff66b 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/spark/storage/BlockStore.scala @@ -1,25 +1,29 @@ package spark.storage -import spark.{Utils, Logging, Serializer, SizeEstimator} -import scala.collection.mutable.ArrayBuffer -import java.io.{File, RandomAccessFile} import java.nio.ByteBuffer -import java.nio.channels.FileChannel.MapMode -import java.util.{UUID, LinkedHashMap} -import java.util.concurrent.Executors -import java.util.concurrent.ConcurrentHashMap -import it.unimi.dsi.fastutil.io._ -import java.util.concurrent.ArrayBlockingQueue + +import spark.Logging /** * Abstract class to store blocks */ -abstract class BlockStore(blockManager: BlockManager) extends Logging { - initLogging() - - def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) - - def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer] +private[spark] +abstract class BlockStore(val blockManager: BlockManager) extends Logging { + def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) + + /** + * Put in a block and, possibly, also return its content as either bytes or another Iterator. + * This is used to efficiently write the values to multiple locations (e.g. for replication). + * + * @return the values put if returnValues is true, or null otherwise + */ + def putValues(blockId: String, values: Iterator[Any], level: StorageLevel, returnValues: Boolean) + : Either[Iterator[Any], ByteBuffer] + + /** + * Return the size of a block. + */ + def getSize(blockId: String): Long def getBytes(blockId: String): Option[ByteBuffer] @@ -27,284 +31,7 @@ abstract class BlockStore(blockManager: BlockManager) extends Logging { def remove(blockId: String) - def dataSerialize(values: Iterator[Any]): ByteBuffer = blockManager.dataSerialize(values) - - def dataDeserialize(bytes: ByteBuffer): Iterator[Any] = blockManager.dataDeserialize(bytes) + def contains(blockId: String): Boolean def clear() { } } - -/** - * Class to store blocks in memory - */ -class MemoryStore(blockManager: BlockManager, maxMemory: Long) - extends BlockStore(blockManager) { - - case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false) - - private val memoryStore = new LinkedHashMap[String, Entry](32, 0.75f, true) - private var currentMemory = 0L - - //private val blockDropper = Executors.newSingleThreadExecutor() - private val blocksToDrop = new ArrayBlockingQueue[String](10000, true) - private val blockDropper = new Thread("memory store - block dropper") { - override def run() { - try{ - while (true) { - val blockId = blocksToDrop.take() - logDebug("Block " + blockId + " ready to be dropped") - blockManager.dropFromMemory(blockId) - } - } catch { - case ie: InterruptedException => - logInfo("Shutting down block dropper") - } - } - } - blockDropper.start() - - def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { - if (level.deserialized) { - bytes.rewind() - val values = dataDeserialize(bytes) - val elements = new ArrayBuffer[Any] - elements ++= values - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - ensureFreeSpace(sizeEstimate) - val entry = new Entry(elements, sizeEstimate, true) - memoryStore.synchronized { memoryStore.put(blockId, entry) } - currentMemory += sizeEstimate - logDebug("Block " + blockId + " stored as values to memory") - } else { - val entry = new Entry(bytes, bytes.array().length, false) - ensureFreeSpace(bytes.array.length) - memoryStore.synchronized { memoryStore.put(blockId, entry) } - currentMemory += bytes.array().length - logDebug("Block " + blockId + " stored as " + bytes.array().length + " bytes to memory") - } - } - - def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer] = { - if (level.deserialized) { - val elements = new ArrayBuffer[Any] - elements ++= values - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - ensureFreeSpace(sizeEstimate) - val entry = new Entry(elements, sizeEstimate, true) - memoryStore.synchronized { memoryStore.put(blockId, entry) } - currentMemory += sizeEstimate - logDebug("Block " + blockId + " stored as values to memory") - return Left(elements.iterator) - } else { - val bytes = dataSerialize(values) - ensureFreeSpace(bytes.array().length) - val entry = new Entry(bytes, bytes.array().length, false) - memoryStore.synchronized { memoryStore.put(blockId, entry) } - currentMemory += bytes.array().length - logDebug("Block " + blockId + " stored as " + bytes.array.length + " bytes to memory") - return Right(bytes) - } - } - - def getBytes(blockId: String): Option[ByteBuffer] = { - throw new UnsupportedOperationException("Not implemented") - } - - def getValues(blockId: String): Option[Iterator[Any]] = { - val entry = memoryStore.synchronized { memoryStore.get(blockId) } - if (entry == null) { - return None - } - if (entry.deserialized) { - return Some(entry.value.asInstanceOf[ArrayBuffer[Any]].toIterator) - } else { - return Some(dataDeserialize(entry.value.asInstanceOf[ByteBuffer])) - } - } - - def remove(blockId: String) { - memoryStore.synchronized { - val entry = memoryStore.get(blockId) - if (entry != null) { - memoryStore.remove(blockId) - currentMemory -= entry.size - logDebug("Block " + blockId + " of size " + entry.size + " dropped from memory") - } else { - logWarning("Block " + blockId + " could not be removed as it doesnt exist") - } - } - } - - override def clear() { - memoryStore.synchronized { - memoryStore.clear() - } - //blockDropper.shutdown() - blockDropper.interrupt() - logInfo("MemoryStore cleared") - } - - private def ensureFreeSpace(space: Long) { - logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( - space, currentMemory, maxMemory)) - - if (maxMemory - currentMemory < space) { - - val selectedBlocks = new ArrayBuffer[String]() - var selectedMemory = 0L - - memoryStore.synchronized { - val iter = memoryStore.entrySet().iterator() - while (maxMemory - (currentMemory - selectedMemory) < space && iter.hasNext) { - val pair = iter.next() - val blockId = pair.getKey - val entry = pair.getValue() - if (!entry.dropPending) { - selectedBlocks += blockId - entry.dropPending = true - } - selectedMemory += pair.getValue.size - logDebug("Block " + blockId + " selected for dropping") - } - } - - logDebug("" + selectedBlocks.size + " new blocks selected for dropping, " + - blocksToDrop.size + " blocks pending") - var i = 0 - while (i < selectedBlocks.size) { - blocksToDrop.add(selectedBlocks(i)) - i += 1 - } - selectedBlocks.clear() - } - } -} - - -/** - * Class to store blocks in disk - */ -class DiskStore(blockManager: BlockManager, rootDirs: String) - extends BlockStore(blockManager) { - - val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - val localDirs = createLocalDirs() - var lastLocalDirUsed = 0 - - addShutdownHook() - - def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { - logDebug("Attempting to put block " + blockId) - val startTime = System.currentTimeMillis - val file = createFile(blockId) - if (file != null) { - val channel = new RandomAccessFile(file, "rw").getChannel() - val buffer = channel.map(MapMode.READ_WRITE, 0, bytes.array.length) - buffer.put(bytes.array) - channel.close() - val finishTime = System.currentTimeMillis - logDebug("Block " + blockId + " stored to file of " + bytes.array.length + " bytes to disk in " + (finishTime - startTime) + " ms") - } else { - logError("File not created for block " + blockId) - } - } - - def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer] = { - val bytes = dataSerialize(values) - logDebug("Converted block " + blockId + " to " + bytes.array.length + " bytes") - putBytes(blockId, bytes, level) - return Right(bytes) - } - - def getBytes(blockId: String): Option[ByteBuffer] = { - val file = getFile(blockId) - val length = file.length().toInt - val channel = new RandomAccessFile(file, "r").getChannel() - val bytes = ByteBuffer.allocate(length) - bytes.put(channel.map(MapMode.READ_WRITE, 0, length)) - return Some(bytes) - } - - def getValues(blockId: String): Option[Iterator[Any]] = { - val file = getFile(blockId) - val length = file.length().toInt - val channel = new RandomAccessFile(file, "r").getChannel() - val bytes = channel.map(MapMode.READ_ONLY, 0, length) - val buffer = dataDeserialize(bytes) - channel.close() - return Some(buffer) - } - - def remove(blockId: String) { - throw new UnsupportedOperationException("Not implemented") - } - - private def createFile(blockId: String): File = { - val file = getFile(blockId) - if (file == null) { - lastLocalDirUsed = (lastLocalDirUsed + 1) % localDirs.size - val newFile = new File(localDirs(lastLocalDirUsed), blockId) - newFile.getParentFile.mkdirs() - return newFile - } else { - logError("File for block " + blockId + " already exists on disk, " + file) - return null - } - } - - private def getFile(blockId: String): File = { - logDebug("Getting file for block " + blockId) - // Search for the file in all the local directories, only one of them should have the file - val files = localDirs.map(localDir => new File(localDir, blockId)).filter(_.exists) - if (files.size > 1) { - throw new Exception("Multiple files for same block " + blockId + " exists: " + - files.map(_.toString).reduceLeft(_ + ", " + _)) - return null - } else if (files.size == 0) { - return null - } else { - logDebug("Got file " + files(0) + " of size " + files(0).length + " bytes") - return files(0) - } - } - - private def createLocalDirs(): Seq[File] = { - logDebug("Creating local directories at root dirs '" + rootDirs + "'") - rootDirs.split("[;,:]").map(rootDir => { - var foundLocalDir: Boolean = false - var localDir: File = null - var localDirUuid: UUID = null - var tries = 0 - while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { - tries += 1 - try { - localDirUuid = UUID.randomUUID() - localDir = new File(rootDir, "spark-local-" + localDirUuid) - if (!localDir.exists) { - localDir.mkdirs() - foundLocalDir = true - } - } catch { - case e: Exception => - logWarning("Attempt " + tries + " to create local dir failed", e) - } - } - if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + - " attempts to create local dir in " + rootDir) - System.exit(1) - } - logDebug("Created local directory at " + localDir) - localDir - }) - } - - private def addShutdownHook() { - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { - logDebug("Shutdown hook called") - localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) - } - }) - } -} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala new file mode 100644 index 0000000000..d0c592ccb1 --- /dev/null +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -0,0 +1,176 @@ +package spark.storage + +import java.nio.ByteBuffer +import java.io.{File, FileOutputStream, RandomAccessFile} +import java.nio.channels.FileChannel.MapMode +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import java.util.{Random, Date} +import spark.Utils +import java.text.SimpleDateFormat + +/** + * Stores BlockManager blocks on disk. + */ +private class DiskStore(blockManager: BlockManager, rootDirs: String) + extends BlockStore(blockManager) { + + val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + + // Create one local directory for each path mentioned in spark.local.dir; then, inside this + // directory, create multiple subdirectories that we will hash files into, in order to avoid + // having really large inodes at the top level. + val localDirs = createLocalDirs() + val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + + addShutdownHook() + + override def getSize(blockId: String): Long = { + getFile(blockId).length() + } + + override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + logDebug("Attempting to put block " + blockId) + val startTime = System.currentTimeMillis + val file = createFile(blockId) + val channel = new RandomAccessFile(file, "rw").getChannel() + while (bytes.remaining > 0) { + channel.write(bytes) + } + channel.close() + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file on disk in %d ms".format( + blockId, Utils.memoryBytesToString(bytes.limit), (finishTime - startTime))) + } + + override def putValues( + blockId: String, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean) + : Either[Iterator[Any], ByteBuffer] = { + + logDebug("Attempting to write values for block " + blockId) + val startTime = System.currentTimeMillis + val file = createFile(blockId) + val fileOut = blockManager.wrapForCompression( + new FastBufferedOutputStream(new FileOutputStream(file))) + val objOut = blockManager.serializer.newInstance().serializeStream(fileOut) + objOut.writeAll(values) + objOut.close() + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file on disk in %d ms".format( + blockId, Utils.memoryBytesToString(file.length()), (finishTime - startTime))) + + if (returnValues) { + // Return a byte buffer for the contents of the file + val channel = new RandomAccessFile(file, "r").getChannel() + val buffer = channel.map(MapMode.READ_ONLY, 0, channel.size()) + channel.close() + Right(buffer) + } else { + null + } + } + + override def getBytes(blockId: String): Option[ByteBuffer] = { + val file = getFile(blockId) + val length = file.length().toInt + val channel = new RandomAccessFile(file, "r").getChannel() + val bytes = channel.map(MapMode.READ_ONLY, 0, length) + channel.close() + Some(bytes) + } + + override def getValues(blockId: String): Option[Iterator[Any]] = { + getBytes(blockId).map(blockManager.dataDeserialize(_)) + } + + override def remove(blockId: String) { + val file = getFile(blockId) + if (file.exists()) { + file.delete() + } + } + + override def contains(blockId: String): Boolean = { + getFile(blockId).exists() + } + + private def createFile(blockId: String): File = { + val file = getFile(blockId) + if (file.exists()) { + throw new Exception("File for block " + blockId + " already exists on disk: " + file) + } + file + } + + private def getFile(blockId: String): File = { + logDebug("Getting file for block " + blockId) + + // Figure out which local directory it hashes to, and which subdirectory in that + val hash = math.abs(blockId.hashCode) + val dirId = hash % localDirs.length + val subDirId = (hash / localDirs.length) % subDirsPerLocalDir + + // Create the subdirectory if it doesn't already exist + var subDir = subDirs(dirId)(subDirId) + if (subDir == null) { + subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) + newDir.mkdir() + subDirs(dirId)(subDirId) = newDir + newDir + } + } + } + + new File(subDir, blockId) + } + + private def createLocalDirs(): Array[File] = { + logDebug("Creating local directories at root dirs '" + rootDirs + "'") + val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + rootDirs.split(",").map(rootDir => { + var foundLocalDir: Boolean = false + var localDir: File = null + var localDirId: String = null + var tries = 0 + val rand = new Random() + while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { + tries += 1 + try { + localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) + localDir = new File(rootDir, "spark-local-" + localDirId) + if (!localDir.exists) { + localDir.mkdirs() + foundLocalDir = true + } + } catch { + case e: Exception => + logWarning("Attempt " + tries + " to create local dir failed", e) + } + } + if (!foundLocalDir) { + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + + " attempts to create local dir in " + rootDir) + System.exit(1) + } + logInfo("Created local directory at " + localDir) + localDir + }) + } + + private def addShutdownHook() { + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { + override def run() { + logDebug("Shutdown hook called") + localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + } + }) + } +} diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala new file mode 100644 index 0000000000..74ef326038 --- /dev/null +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -0,0 +1,217 @@ +package spark.storage + +import java.util.LinkedHashMap +import java.util.concurrent.ArrayBlockingQueue +import spark.{SizeEstimator, Utils} +import java.nio.ByteBuffer +import collection.mutable.ArrayBuffer + +/** + * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as + * serialized ByteBuffers. + */ +private class MemoryStore(blockManager: BlockManager, maxMemory: Long) + extends BlockStore(blockManager) { + + case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false) + + private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true) + private var currentMemory = 0L + + logInfo("MemoryStore started with capacity %s.".format(Utils.memoryBytesToString(maxMemory))) + + def freeMemory: Long = maxMemory - currentMemory + + override def getSize(blockId: String): Long = { + synchronized { + entries.get(blockId).size + } + } + + override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + if (level.deserialized) { + bytes.rewind() + val values = blockManager.dataDeserialize(bytes) + val elements = new ArrayBuffer[Any] + elements ++= values + val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) + tryToPut(blockId, elements, sizeEstimate, true) + } else { + val entry = new Entry(bytes, bytes.limit, false) + ensureFreeSpace(blockId, bytes.limit) + synchronized { entries.put(blockId, entry) } + tryToPut(blockId, bytes, bytes.limit, false) + } + } + + override def putValues( + blockId: String, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean) + : Either[Iterator[Any], ByteBuffer] = { + + if (level.deserialized) { + val elements = new ArrayBuffer[Any] + elements ++= values + val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) + tryToPut(blockId, elements, sizeEstimate, true) + Left(elements.iterator) + } else { + val bytes = blockManager.dataSerialize(values) + tryToPut(blockId, bytes, bytes.limit, false) + Right(bytes) + } + } + + override def getBytes(blockId: String): Option[ByteBuffer] = { + val entry = synchronized { + entries.get(blockId) + } + if (entry == null) { + None + } else if (entry.deserialized) { + Some(blockManager.dataSerialize(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) + } else { + Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data + } + } + + override def getValues(blockId: String): Option[Iterator[Any]] = { + val entry = synchronized { + entries.get(blockId) + } + if (entry == null) { + None + } else if (entry.deserialized) { + Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + } else { + val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data + Some(blockManager.dataDeserialize(buffer)) + } + } + + override def remove(blockId: String) { + synchronized { + val entry = entries.get(blockId) + if (entry != null) { + entries.remove(blockId) + currentMemory -= entry.size + logInfo("Block %s of size %d dropped from memory (free %d)".format( + blockId, entry.size, freeMemory)) + } else { + logWarning("Block " + blockId + " could not be removed as it does not exist") + } + } + } + + override def clear() { + synchronized { + entries.clear() + } + logInfo("MemoryStore cleared") + } + + /** + * Return the RDD ID that a given block ID is from, or null if it is not an RDD block. + */ + private def getRddId(blockId: String): String = { + if (blockId.startsWith("rdd_")) { + blockId.split('_')(1) + } else { + null + } + } + + /** + * Try to put in a set of values, if we can free up enough space. The value should either be + * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) + * size must also be passed by the caller. + */ + private def tryToPut(blockId: String, value: Any, size: Long, deserialized: Boolean): Boolean = { + synchronized { + if (ensureFreeSpace(blockId, size)) { + val entry = new Entry(value, size, deserialized) + entries.put(blockId, entry) + currentMemory += size + if (deserialized) { + logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format( + blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory))) + } else { + logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( + blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory))) + } + true + } else { + // Tell the block manager that we couldn't put it in memory so that it can drop it to + // disk if the block allows disk storage. + val data = if (deserialized) { + Left(value.asInstanceOf[ArrayBuffer[Any]].iterator) + } else { + Right(value.asInstanceOf[ByteBuffer].duplicate()) + } + blockManager.dropFromMemory(blockId, data) + false + } + } + } + + /** + * Tries to free up a given amount of space to store a particular block, but can fail and return + * false if either the block is bigger than our memory or it would require replacing another + * block from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that + * don't fit into memory that we want to avoid). + * + * Assumes that a lock on entries is held by the caller. + */ + private def ensureFreeSpace(blockIdToAdd: String, space: Long): Boolean = { + logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( + space, currentMemory, maxMemory)) + + if (space > maxMemory) { + logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") + return false + } + + if (maxMemory - currentMemory < space) { + val rddToAdd = getRddId(blockIdToAdd) + val selectedBlocks = new ArrayBuffer[String]() + var selectedMemory = 0L + + val iterator = entries.entrySet().iterator() + while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { + val pair = iterator.next() + val blockId = pair.getKey + if (rddToAdd != null && rddToAdd == getRddId(blockId)) { + logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " + + "block from the same RDD") + return false + } + selectedBlocks += blockId + selectedMemory += pair.getValue.size + } + + if (maxMemory - (currentMemory - selectedMemory) >= space) { + logInfo(selectedBlocks.size + " blocks selected for dropping") + for (blockId <- selectedBlocks) { + val entry = entries.get(blockId) + val data = if (entry.deserialized) { + Left(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + } else { + Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) + } + blockManager.dropFromMemory(blockId, data) + } + return true + } else { + return false + } + } + return true + } + + override def contains(blockId: String): Boolean = { + synchronized { entries.containsKey(blockId) } + } +} + diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index f067a2a6c5..2237ce92b3 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -1,8 +1,8 @@ package spark.storage -import java.io._ +import java.io.{Externalizable, ObjectInput, ObjectOutput} -class StorageLevel( +private[spark] class StorageLevel( var useDisk: Boolean, var useMemory: Boolean, var deserialized: Boolean, @@ -63,15 +63,16 @@ class StorageLevel( "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) } -object StorageLevel { +private[spark] object StorageLevel { val NONE = new StorageLevel(false, false, false) val DISK_ONLY = new StorageLevel(true, false, false) - val MEMORY_ONLY = new StorageLevel(false, true, false) - val MEMORY_ONLY_2 = new StorageLevel(false, true, false, 2) - val MEMORY_ONLY_DESER = new StorageLevel(false, true, true) - val MEMORY_ONLY_DESER_2 = new StorageLevel(false, true, true, 2) - val DISK_AND_MEMORY = new StorageLevel(true, true, false) - val DISK_AND_MEMORY_2 = new StorageLevel(true, true, false, 2) - val DISK_AND_MEMORY_DESER = new StorageLevel(true, true, true) - val DISK_AND_MEMORY_DESER_2 = new StorageLevel(true, true, true, 2) + val DISK_ONLY_2 = new StorageLevel(true, false, false, 2) + val MEMORY_ONLY = new StorageLevel(false, true, true) + val MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2) + val MEMORY_ONLY_SER = new StorageLevel(false, true, false) + val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2) + val MEMORY_AND_DISK = new StorageLevel(true, true, true) + val MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2) + val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) + val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 57d212e4ca..f670ccb709 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -17,7 +17,7 @@ import java.util.concurrent.TimeoutException /** * Various utility classes for working with Akka. */ -object AkkaUtils { +private[spark] object AkkaUtils { /** * Creates an ActorSystem ready for remoting, with various Spark features. Returns both the * ActorSystem itself and its port (which is hard to get from Akka). @@ -31,6 +31,8 @@ object AkkaUtils { akka.remote.netty.hostname = "%s" akka.remote.netty.port = %d akka.remote.netty.connection-timeout = 1s + akka.remote.netty.execution-pool-size = 8 + akka.actor.default-dispatcher.throughput = 30 """.format(host, port)) val actorSystem = ActorSystem("spark", akkaConf, getClass.getClassLoader) diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala index 0ce255105a..d7e67497fe 100644 --- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala @@ -2,10 +2,19 @@ package spark.util import java.io.InputStream import java.nio.ByteBuffer +import spark.storage.BlockManager + +/** + * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose() + * at the end of the stream (e.g. to close a memory-mapped file). + */ +private[spark] +class ByteBufferInputStream(private var buffer: ByteBuffer, dispose: Boolean = false) + extends InputStream { -class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream { override def read(): Int = { - if (buffer.remaining() == 0) { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() -1 } else { buffer.get() & 0xFF @@ -17,7 +26,8 @@ class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream { } override def read(dest: Array[Byte], offset: Int, length: Int): Int = { - if (buffer.remaining() == 0) { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() -1 } else { val amountToGet = math.min(buffer.remaining(), length) @@ -27,8 +37,27 @@ class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream { } override def skip(bytes: Long): Long = { - val amountToSkip = math.min(bytes, buffer.remaining).toInt - buffer.position(buffer.position + amountToSkip) - return amountToSkip + if (buffer != null) { + val amountToSkip = math.min(bytes, buffer.remaining).toInt + buffer.position(buffer.position + amountToSkip) + if (buffer.remaining() == 0) { + cleanUp() + } + amountToSkip + } else { + 0L + } + } + + /** + * Clean up the buffer, and potentially dispose of it using BlockManager.dispose(). + */ + private def cleanUp() { + if (buffer != null) { + if (dispose) { + BlockManager.dispose(buffer) + } + buffer = null + } } } diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala index c3ff063569..0427646747 100644 --- a/core/src/main/scala/spark/util/IntParam.scala +++ b/core/src/main/scala/spark/util/IntParam.scala @@ -3,7 +3,7 @@ package spark.util /** * An extractor object for parsing strings into integers. */ -object IntParam { +private[spark] object IntParam { def unapply(str: String): Option[Int] = { try { Some(str.toInt) diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala index 4fba914afe..3726738842 100644 --- a/core/src/main/scala/spark/util/MemoryParam.scala +++ b/core/src/main/scala/spark/util/MemoryParam.scala @@ -6,7 +6,7 @@ import spark.Utils * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing * the number of megabytes. Supports the same formats as Utils.memoryStringToMb. */ -object MemoryParam { +private[spark] object MemoryParam { def unapply(str: String): Option[Int] = { try { Some(Utils.memoryStringToMb(str)) diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala index 0830843a77..09d588fe1c 100644 --- a/core/src/main/scala/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/spark/util/SerializableBuffer.scala @@ -8,6 +8,7 @@ import java.nio.channels.Channels * A wrapper around a java.nio.ByteBuffer that is serializable through Java serialization, to make * it easier to pass ByteBuffers in case class messages. */ +private[spark] class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable { def value = buffer diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 11d7939204..023ec09332 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -5,6 +5,7 @@ package spark.util * numerically robust way. Includes support for merging two StatCounters. Based on Welford and * Chan's algorithms described at http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance. */ +private[spark] class StatCounter(values: TraversableOnce[Double]) extends Serializable { private var n: Long = 0 // Running count of our values private var mu: Double = 0 // Running mean of our values @@ -82,7 +83,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { } } -object StatCounter { +private[spark] object StatCounter { def apply(values: TraversableOnce[Double]) = new StatCounter(values) def apply(values: Double*) = new StatCounter(values) diff --git a/core/src/main/twirl/common/layout.scala.html b/core/src/main/twirl/spark/deploy/common/layout.scala.html index b9192060aa..b9192060aa 100644 --- a/core/src/main/twirl/common/layout.scala.html +++ b/core/src/main/twirl/spark/deploy/common/layout.scala.html diff --git a/core/src/main/twirl/masterui/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html index 784d692fc2..784d692fc2 100644 --- a/core/src/main/twirl/masterui/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html diff --git a/core/src/main/twirl/masterui/executors_table.scala.html b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html index cafc42c80e..cafc42c80e 100644 --- a/core/src/main/twirl/masterui/executors_table.scala.html +++ b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html diff --git a/core/src/main/twirl/masterui/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html index 31ca8f4132..7562076b00 100644 --- a/core/src/main/twirl/masterui/index.scala.html +++ b/core/src/main/twirl/spark/deploy/master/index.scala.html @@ -1,13 +1,13 @@ @(state: spark.deploy.MasterState) @import spark.deploy.master._ -@common.html.layout(title = "Spark Master on " + state.uri) { +@spark.deploy.common.html.layout(title = "Spark Master on " + state.uri) { <!-- Cluster Details --> <div class="row"> <div class="span12"> <ul class="unstyled"> - <li><strong>URI:</strong> spark://@(state.uri)</li> + <li><strong>URL:</strong> spark://@(state.uri)</li> <li><strong>Number of Workers:</strong> @state.workers.size </li> <li><strong>Cores:</strong> @state.workers.map(_.cores).sum Total, @state.workers.map(_.coresUsed).sum Used</li> <li><strong>Memory:</strong> @state.workers.map(_.memory).sum Total, @state.workers.map(_.memoryUsed).sum Used</li> @@ -47,4 +47,4 @@ </div> </div> -}
\ No newline at end of file +} diff --git a/core/src/main/twirl/masterui/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html index 73cefb8269..dcf41c28f2 100644 --- a/core/src/main/twirl/masterui/job_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/job_details.scala.html @@ -1,6 +1,6 @@ @(job: spark.deploy.master.JobInfo) -@common.html.layout(title = "Job Details") { +@spark.deploy.common.html.layout(title = "Job Details") { <!-- Job Details --> <div class="row"> @@ -37,4 +37,4 @@ </div> </div> -}
\ No newline at end of file +} diff --git a/core/src/main/twirl/masterui/job_row.scala.html b/core/src/main/twirl/spark/deploy/master/job_row.scala.html index 7c4865bb6e..7c4865bb6e 100644 --- a/core/src/main/twirl/masterui/job_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/job_row.scala.html diff --git a/core/src/main/twirl/masterui/job_table.scala.html b/core/src/main/twirl/spark/deploy/master/job_table.scala.html index 52bad6c4b8..52bad6c4b8 100644 --- a/core/src/main/twirl/masterui/job_table.scala.html +++ b/core/src/main/twirl/spark/deploy/master/job_table.scala.html diff --git a/core/src/main/twirl/masterui/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html index b21bd9c977..017cc4859e 100644 --- a/core/src/main/twirl/masterui/worker_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html @@ -4,7 +4,8 @@ <td> <a href="http://@worker.host:@worker.webUiPort">@worker.id</href> </td> - <td>@worker.host:@worker.port</td> + <td>@{worker.host}:@{worker.port}</td> <td>@worker.cores (@worker.coresUsed Used)</td> - <td>@worker.memory (@worker.memoryUsed Used)</td> -</tr>
\ No newline at end of file + <td>@{spark.Utils.memoryMegabytesToString(worker.memory)} + (@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</td> +</tr> diff --git a/core/src/main/twirl/masterui/worker_table.scala.html b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html index 2028842297..2028842297 100644 --- a/core/src/main/twirl/masterui/worker_table.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html diff --git a/core/src/main/twirl/workerui/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html index c3842dbf85..c3842dbf85 100644 --- a/core/src/main/twirl/workerui/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html diff --git a/core/src/main/twirl/workerui/executors_table.scala.html b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html index 327a2399c7..327a2399c7 100644 --- a/core/src/main/twirl/workerui/executors_table.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html diff --git a/core/src/main/twirl/workerui/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html index edd82e02f2..69746ed02c 100644 --- a/core/src/main/twirl/workerui/index.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html @@ -1,6 +1,6 @@ @(worker: spark.deploy.WorkerState) -@common.html.layout(title = "Spark Worker on " + worker.uri) { +@spark.deploy.common.html.layout(title = "Spark Worker on " + worker.uri) { <!-- Worker Details --> <div class="row"> @@ -12,7 +12,8 @@ (WebUI at <a href="@worker.masterWebUiUrl">@worker.masterWebUiUrl</a>) </li> <li><strong>Cores:</strong> @worker.cores (@worker.coresUsed Used)</li> - <li><strong>Memory:</strong> @worker.memory (@worker.memoryUsed Used)</li> + <li><strong>Memory:</strong> @{spark.Utils.memoryMegabytesToString(worker.memory)} + (@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</li> </ul> </div> </div> @@ -39,4 +40,4 @@ </div> </div> -}
\ No newline at end of file +} diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 02fe16866e..4c99e450bc 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -1,8 +1,10 @@ # Set everything to be logged to the console -log4j.rootCategory=WARN, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=spark-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar Binary files differnew file mode 100644 index 0000000000..e126001c1c --- /dev/null +++ b/core/src/test/resources/uncommons-maths-1.2.2.jar diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d55969c261..71df5941e5 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -56,7 +56,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] { def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = { t1 ++= t2 @@ -71,7 +70,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - test ("value not readable in tasks") { import SetAccum._ val maxI = 1000 @@ -89,4 +87,29 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } + test ("collection accumulators") { + val maxI = 1000 + for (nThreads <- List(1, 10)) { + // test single & multi-threaded + val sc = new SparkContext("local[" + nThreads + "]", "test") + val setAcc = sc.accumulableCollection(mutable.HashSet[Int]()) + val bufferAcc = sc.accumulableCollection(mutable.ArrayBuffer[Int]()) + val mapAcc = sc.accumulableCollection(mutable.HashMap[Int,String]()) + val d = sc.parallelize((1 to maxI) ++ (1 to maxI)) + d.foreach { + x => {setAcc += x; bufferAcc += x; mapAcc += (x -> x.toString)} + } + + // Note that this is typed correctly -- no casts necessary + setAcc.value.size should be (maxI) + bufferAcc.value.size should be (2 * maxI) + mapAcc.value.size should be (maxI) + for (i <- 1 to maxI) { + setAcc.value should contain(i) + bufferAcc.value should contain(i) + mapAcc.value should contain (i -> i.toString) + } + sc.stop() + } + } } diff --git a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala index dff2970566..e00e8c4123 100644 --- a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala +++ b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala @@ -3,6 +3,7 @@ package spark import org.scalatest.FunSuite import org.scalatest.PrivateMethodTester +// TODO: Replace this with a test of MemoryStore class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester { test("constructor test") { val cache = new BoundedMemoryCache(60) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala new file mode 100644 index 0000000000..48c0a830e0 --- /dev/null +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -0,0 +1,140 @@ +package spark + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer + +import SparkContext._ +import storage.StorageLevel + +class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { + + val clusterUrl = "local-cluster[2,1,512]" + + @transient var sc: SparkContext = _ + + after { + if (sc != null) { + sc.stop() + sc = null + } + } + + test("local-cluster format") { + sc = new SparkContext("local-cluster[2,1,512]", "test") + assert(sc.parallelize(1 to 2, 2).count == 2) + sc.stop() + sc = new SparkContext("local-cluster[2 , 1 , 512]", "test") + assert(sc.parallelize(1 to 2, 2).count == 2) + sc.stop() + sc = new SparkContext("local-cluster[2, 1, 512]", "test") + assert(sc.parallelize(1 to 2, 2).count == 2) + sc.stop() + sc = new SparkContext("local-cluster[ 2, 1, 512 ]", "test") + assert(sc.parallelize(1 to 2, 2).count == 2) + sc.stop() + sc = null + } + + test("simple groupByKey") { + sc = new SparkContext(clusterUrl, "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 5) + val groups = pairs.groupByKey(5).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("accumulators") { + sc = new SparkContext(clusterUrl, "test") + val accum = sc.accumulator(0) + sc.parallelize(1 to 10, 10).foreach(x => accum += x) + assert(accum.value === 55) + } + + test("broadcast variables") { + sc = new SparkContext(clusterUrl, "test") + val array = new Array[Int](100) + val bv = sc.broadcast(array) + array(2) = 3 // Change the array -- this should not be seen on workers + val rdd = sc.parallelize(1 to 10, 10) + val sum = rdd.map(x => bv.value.sum).reduce(_ + _) + assert(sum === 0) + } + + test("repeatedly failing task") { + sc = new SparkContext(clusterUrl, "test") + val accum = sc.accumulator(0) + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("more than 4 times")) + } + + test("caching") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).cache() + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching on disk") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching in memory, replicated") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_2) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching in memory, serialized, replicated") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_SER_2) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching on disk, replicated") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY_2) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching in memory and disk, replicated") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_2) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } + + test("caching in memory and disk, serialized, replicated") { + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) + assert(data.count() === 1000) + assert(data.count() === 1000) + assert(data.count() === 1000) + } +} diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala new file mode 100644 index 0000000000..fd7a7bd589 --- /dev/null +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -0,0 +1,95 @@ +package spark + +import com.google.common.io.Files +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter +import java.io.{File, PrintWriter, FileReader, BufferedReader} +import SparkContext._ + +class FileServerSuite extends FunSuite with BeforeAndAfter { + + @transient var sc: SparkContext = _ + @transient var tmpFile : File = _ + @transient var testJarFile : File = _ + + before { + // Create a sample text file + val tmpdir = new File(Files.createTempDir(), "test") + tmpdir.mkdir() + tmpFile = new File(tmpdir, "FileServerSuite.txt") + val pw = new PrintWriter(tmpFile) + pw.println("100") + pw.close() + } + + after { + if (sc != null) { + sc.stop() + sc = null + } + // Clean up downloaded file + if (tmpFile.exists) { + tmpFile.delete() + } + } + + test("Distributing files locally") { + sc = new SparkContext("local[4]", "test") + sc.addFile(tmpFile.toString) + val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) + val result = sc.parallelize(testData).reduceByKey { + val in = new BufferedReader(new FileReader("FileServerSuite.txt")) + val fileVal = in.readLine().toInt + in.close() + _ * fileVal + _ * fileVal + }.collect + println(result) + assert(result.toSet === Set((1,200), (2,300), (3,500))) + } + + test ("Dynamically adding JARS locally") { + sc = new SparkContext("local[4]", "test") + val sampleJarFile = getClass().getClassLoader().getResource("uncommons-maths-1.2.2.jar").getFile() + sc.addJar(sampleJarFile) + val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) + val result = sc.parallelize(testData).reduceByKey { (x,y) => + val fac = Thread.currentThread.getContextClassLoader() + .loadClass("org.uncommons.maths.Maths") + .getDeclaredMethod("factorial", classOf[Int]) + val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt + val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt + a + b + }.collect() + assert(result.toSet === Set((1,2), (2,7), (3,121))) + } + + test("Distributing files on a standalone cluster") { + sc = new SparkContext("local-cluster[1,1,512]", "test") + sc.addFile(tmpFile.toString) + val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) + val result = sc.parallelize(testData).reduceByKey { + val in = new BufferedReader(new FileReader("FileServerSuite.txt")) + val fileVal = in.readLine().toInt + in.close() + _ * fileVal + _ * fileVal + }.collect + println(result) + assert(result.toSet === Set((1,200), (2,300), (3,500))) + } + + test ("Dynamically adding JARS on a standalone cluster") { + sc = new SparkContext("local-cluster[1,1,512]", "test") + val sampleJarFile = getClass().getClassLoader().getResource("uncommons-maths-1.2.2.jar").getFile() + sc.addJar(sampleJarFile) + val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) + val result = sc.parallelize(testData).reduceByKey { (x,y) => + val fac = Thread.currentThread.getContextClassLoader() + .loadClass("org.uncommons.maths.Maths") + .getDeclaredMethod("factorial", classOf[Int]) + val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt + val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt + a + b + }.collect() + assert(result.toSet === Set((1,2), (2,7), (3,121))) + } +} diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index 17c7a8de43..5c1577ed0b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -1,6 +1,6 @@ package spark -import java.io.File +import java.io.{FileWriter, PrintWriter, File} import scala.io.Source @@ -142,4 +142,18 @@ class FileSuite extends FunSuite with BeforeAndAfter { sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } + + test("file caching") { + sc = new SparkContext("local", "test") + val tempDir = Files.createTempDir() + val out = new FileWriter(tempDir + "/input") + out.write("Hello world!\n") + out.write("What's up?\n") + out.write("Goodbye\n") + out.close() + val rdd = sc.textFile(tempDir + "/input").cache() + assert(rdd.count() === 3) + assert(rdd.count() === 3) + assert(rdd.count() === 3) + } } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 24bf021710..5875506179 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -112,15 +112,19 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2)); } + static int foreachCalls = 0; + @Test public void foreach() { + foreachCalls = 0; JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreach(new VoidFunction<String>() { @Override public void call(String s) { - System.out.println(s); + foreachCalls++; } }); + Assert.assertEquals(2, foreachCalls); } @Test diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index 426652dc15..add5221e30 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -22,7 +22,6 @@ class PipedRDDSuite extends FunSuite with BeforeAndAfter { val piped = nums.pipe(Seq("cat")) val c = piped.collect() - println(c.toSeq) assert(c.size === 4) assert(c(0) === "1") assert(c(1) === "2") diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ba9b36adb7..ade457c0f9 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -29,6 +29,11 @@ class RDDSuite extends FunSuite with BeforeAndAfter { assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) + + val partitionSumsWithSplit = nums.mapPartitionsWithSplit { + case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + } + assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) } test("SparkContext.union") { @@ -66,4 +71,43 @@ class RDDSuite extends FunSuite with BeforeAndAfter { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).flatMap(x => 1 to x).checkpoint() assert(rdd.collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) } + + test("basic caching") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(rdd.collect().toList === List(1, 2, 3, 4)) + } + + test("coalesced RDDs") { + sc = new SparkContext("local", "test") + val data = sc.parallelize(1 to 10, 10) + + val coalesced1 = new CoalescedRDD(data, 2) + assert(coalesced1.collect().toList === (1 to 10).toList) + assert(coalesced1.glom().collect().map(_.toList).toList === + List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10))) + + // Check that the narrow dependency is also specified correctly + assert(coalesced1.dependencies.head.getParents(0).toList === List(0, 1, 2, 3, 4)) + assert(coalesced1.dependencies.head.getParents(1).toList === List(5, 6, 7, 8, 9)) + + val coalesced2 = new CoalescedRDD(data, 3) + assert(coalesced2.collect().toList === (1 to 10).toList) + assert(coalesced2.glom().collect().map(_.toList).toList === + List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10))) + + val coalesced3 = new CoalescedRDD(data, 10) + assert(coalesced3.collect().toList === (1 to 10).toList) + assert(coalesced3.glom().collect().map(_.toList).toList === + (1 to 10).map(x => List(x)).toList) + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = new CoalescedRDD(data, 20) + assert(coalesced4.collect().toList === (1 to 10).toList) + assert(coalesced4.glom().collect().map(_.toList).toList === + (1 to 10).map(x => List(x)).toList) + } } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index f622c413f7..90760b8a85 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -15,16 +15,16 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { - + var sc: SparkContext = _ - + after { if (sc != null) { sc.stop() sc = null } } - + test("groupByKey") { sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) @@ -57,7 +57,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { val valuesFor2 = groups.find(_._1 == 2).get._2 assert(valuesFor2.toList.sorted === List(1)) } - + test("groupByKey with many output partitions") { sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) @@ -69,6 +69,22 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { assert(valuesFor2.toList.sorted === List(1)) } + test("groupByKey with compression") { + try { + System.setProperty("spark.blockManager.compress", "true") + sc = new SparkContext("local", "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) + val groups = pairs.groupByKey(4).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } finally { + System.setProperty("spark.blockManager.compress", "false") + } + } + test("reduceByKey") { sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) @@ -187,7 +203,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { (4, (ArrayBuffer(), ArrayBuffer('w'))) )) } - + test("zero-partition RDD") { sc = new SparkContext("local", "test") val emptyDir = Files.createTempDir() @@ -195,7 +211,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { assert(file.splits.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) } test("map-side combine") { @@ -212,7 +228,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { _+_, _+_, false) - val shuffledRdd = new ShuffledRDD( + val shuffledRdd = new ShuffledAggregatedRDD( pairs, aggregator, new HashPartitioner(2)) assert(shuffledRdd.collect().toSet === Set((1, 8), (2, 1))) @@ -220,7 +236,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { // not see an exception because mergeCombine should not have been called. val aggregatorWithException = new Aggregator[Int, Int, Int]( (v: Int) => v, _+_, ShuffleSuite.mergeCombineException, false) - val shuffledRdd1 = new ShuffledRDD( + val shuffledRdd1 = new ShuffledAggregatedRDD( pairs, aggregatorWithException, new HashPartitioner(2)) assert(shuffledRdd1.collect().toSet === Set((1, 8), (2, 1))) @@ -228,7 +244,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { // expect to see an exception thrown. val aggregatorWithException1 = new Aggregator[Int, Int, Int]( (v: Int) => v, _+_, ShuffleSuite.mergeCombineException) - val shuffledRdd2 = new ShuffledRDD( + val shuffledRdd2 = new ShuffledAggregatedRDD( pairs, aggregatorWithException1, new HashPartitioner(2)) evaluating { shuffledRdd2.collect() } should produce [SparkException] } diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index 8fa1442a4d..c87595ecb3 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -17,7 +17,7 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with test("sortByKey") { sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0))) + val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) } @@ -25,18 +25,56 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr) - assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey() + assert(sorted.splits.size === 2) + assert(sorted.collect() === pairArr.sortBy(_._1)) } + test("large array with one split") { + sc = new SparkContext("local", "test") + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey(true, 1) + assert(sorted.splits.size === 1) + assert(sorted.collect() === pairArr.sortBy(_._1)) + } + + test("large array with many splits") { + sc = new SparkContext("local", "test") + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey(true, 20) + assert(sorted.splits.size === 20) + assert(sorted.collect() === pairArr.sortBy(_._1)) + } + test("sort descending") { sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr) + val pairs = sc.parallelize(pairArr, 2) assert(pairs.sortByKey(false).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) } + test("sort descending with one split") { + sc = new SparkContext("local", "test") + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 1) + assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) + } + + test("sort descending with many splits") { + sc = new SparkContext("local", "test") + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + assert(pairs.sortByKey(false, 20).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) + } + test("more partitions than elements") { sc = new SparkContext("local", "test") val rand = new scala.util.Random() @@ -48,7 +86,7 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with test("empty RDD") { sc = new SparkContext("local", "test") val pairArr = new Array[(Int, Int)](0) - val pairs = sc.parallelize(pairArr) + val pairs = sc.parallelize(pairArr, 2) assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) } @@ -58,11 +96,11 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with val sorted = sc.parallelize(pairArr, 4).sortByKey() assert(sorted.collect() === pairArr.sortBy(_._1)) val partitions = sorted.collectPartitions() - logInfo("partition lengths: " + partitions.map(_.length).mkString(", ")) - partitions(0).length should be > 200 - partitions(1).length should be > 200 - partitions(2).length should be > 200 - partitions(3).length should be > 200 + logInfo("Partition lengths: " + partitions.map(_.length).mkString(", ")) + partitions(0).length should be > 180 + partitions(1).length should be > 180 + partitions(2).length should be > 180 + partitions(3).length should be > 180 partitions(0).last should be < partitions(1).head partitions(1).last should be < partitions(2).head partitions(2).last should be < partitions(3).head @@ -75,10 +113,10 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with assert(sorted.collect() === pairArr.sortBy(_._1).reverse) val partitions = sorted.collectPartitions() logInfo("partition lengths: " + partitions.map(_.length).mkString(", ")) - partitions(0).length should be > 200 - partitions(1).length should be > 200 - partitions(2).length should be > 200 - partitions(3).length should be > 200 + partitions(0).length should be > 180 + partitions(1).length should be > 180 + partitions(2).length should be > 180 + partitions(3).length should be > 180 partitions(0).last should be > partitions(1).head partitions(1).last should be > partitions(2).head partitions(2).last should be > partitions(3).head diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index f3f891e471..f61fd45ed3 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -55,9 +55,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a3 = new Array[Byte](400) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_DESER) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_DESER, false) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, false) // Checking whether blocks are in memory assert(store.getSingle("a1") != None, "a1 was not in store") @@ -69,9 +69,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(master.mustGetLocations(GetLocations("a2")).size > 0, "master was not told about a2") assert(master.mustGetLocations(GetLocations("a3")).size === 0, "master was told about a3") - // Setting storage level of a1 and a2 to invalid; they should be removed from store and master - store.setLevel("a1", new StorageLevel(false, false, false, 1)) - store.setLevel("a2", new StorageLevel(true, false, false, 0)) + // Drop a1 and a2 from memory; this should be reported back to the master + store.dropFromMemory("a1", null) + store.dropFromMemory("a2", null) assert(store.getSingle("a1") === None, "a1 not removed from store") assert(store.getSingle("a2") === None, "a2 not removed from store") assert(master.mustGetLocations(GetLocations("a1")).size === 0, "master did not remove a1") @@ -83,16 +83,16 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_DESER) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_DESER) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") != None, "a3 was not in store") Thread.sleep(100) assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2") != None, "a2 was not in store") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") != None, "a1 was not in store") assert(store.getSingle("a2") != None, "a2 was not in store") Thread.sleep(100) @@ -104,22 +104,67 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) Thread.sleep(100) assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") != None, "a3 was not in store") assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2") != None, "a2 was not in store") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) Thread.sleep(100) assert(store.getSingle("a1") != None, "a1 was not in store") assert(store.getSingle("a2") != None, "a2 was not in store") - assert(store.getSingle("a3") === None, "a1 was in store") + assert(store.getSingle("a3") === None, "a3 was in store") } - + + test("in-memory LRU for partitions of same RDD") { + val store = new BlockManager(master, new KryoSerializer, 1200) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY) + Thread.sleep(100) + // Even though we accessed rdd_0_3 last, it should not have replaced partitiosn 1 and 2 + // from the same RDD + assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store") + assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store") + assert(store.getSingle("rdd_0_1") != None, "rdd_0_1 was not in store") + // Check that rdd_0_3 doesn't replace them even after further accesses + assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store") + assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store") + assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store") + } + + test("in-memory LRU for partitions of multiple RDDs") { + val store = new BlockManager(master, new KryoSerializer, 1200) + store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) + Thread.sleep(100) + // At this point rdd_1_1 should've replaced rdd_0_1 + assert(store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was not in store") + assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store") + assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store") + // Do a get() on rdd_0_2 so that it is the most recently used item + assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store") + // Put in more partitions from RDD 0; they should replace rdd_1_1 + store.putSingle("rdd_0_3", new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_4", new Array[Byte](400), StorageLevel.MEMORY_ONLY) + Thread.sleep(100) + // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped + // when we try to add rdd_0_4. + assert(!store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was in store") + assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store") + assert(!store.memoryStore.contains("rdd_0_4"), "rdd_0_4 was in store") + assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store") + assert(store.memoryStore.contains("rdd_0_3"), "rdd_0_3 was not in store") + } + test("on-disk storage") { val store = new BlockManager(master, new KryoSerializer, 1200) val a1 = new Array[Byte](400) @@ -128,9 +173,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("a1", a1, StorageLevel.DISK_ONLY) store.putSingle("a2", a2, StorageLevel.DISK_ONLY) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) - assert(store.getSingle("a2") != None, "a2 was not in store") - assert(store.getSingle("a3") != None, "a3 was not in store") - assert(store.getSingle("a1") != None, "a1 was not in store") + assert(store.getSingle("a2") != None, "a2 was in store") + assert(store.getSingle("a3") != None, "a3 was in store") + assert(store.getSingle("a1") != None, "a1 was in store") } test("disk and memory storage") { @@ -138,13 +183,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.DISK_AND_MEMORY_DESER) - store.putSingle("a2", a2, StorageLevel.DISK_AND_MEMORY_DESER) - store.putSingle("a3", a3, StorageLevel.DISK_AND_MEMORY_DESER) + store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) + store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) Thread.sleep(100) assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") != None, "a3 was not in store") + assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") assert(store.getSingle("a1") != None, "a1 was not in store") + assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store") + } + + test("disk and memory storage with getLocalBytes") { + val store = new BlockManager(master, new KryoSerializer, 1200) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) + store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) + Thread.sleep(100) + assert(store.getLocalBytes("a2") != None, "a2 was not in store") + assert(store.getLocalBytes("a3") != None, "a3 was not in store") + assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") + assert(store.getLocalBytes("a1") != None, "a1 was not in store") + assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store") } test("disk and memory storage with serialization") { @@ -152,13 +215,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.DISK_AND_MEMORY) - store.putSingle("a2", a2, StorageLevel.DISK_AND_MEMORY) - store.putSingle("a3", a3, StorageLevel.DISK_AND_MEMORY) + store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) + store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) + store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) Thread.sleep(100) assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") != None, "a3 was not in store") + assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") assert(store.getSingle("a1") != None, "a1 was not in store") + assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store") + } + + test("disk and memory storage with serialization and getLocalBytes") { + val store = new BlockManager(master, new KryoSerializer, 1200) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) + store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) + store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) + Thread.sleep(100) + assert(store.getLocalBytes("a2") != None, "a2 was not in store") + assert(store.getLocalBytes("a3") != None, "a3 was not in store") + assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") + assert(store.getLocalBytes("a1") != None, "a1 was not in store") + assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store") } test("LRU with mixed storage levels") { @@ -168,8 +249,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a3 = new Array[Byte](400) val a4 = new Array[Byte](400) // First store a1 and a2, both in memory, and a3, on disk only - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk assert(store.getSingle("a1") != None, "a2 was not in store") @@ -179,7 +260,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.getSingle("a2") != None, "a3 was not in store") assert(store.getSingle("a3") != None, "a1 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out - store.putSingle("a4", a4, StorageLevel.DISK_AND_MEMORY) + store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) Thread.sleep(100) assert(store.getSingle("a1") == None, "a1 was in store") assert(store.getSingle("a2") != None, "a2 was not in store") @@ -192,9 +273,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_DESER) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_DESER) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY_DESER) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY) + store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY) Thread.sleep(100) assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) @@ -204,7 +285,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_DESER) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY) Thread.sleep(100) assert(store.get("list1") != None, "list1 was not in store") assert(store.get("list1").get.size == 2) @@ -220,8 +301,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list3 = List(new Array[Byte](200), new Array[Byte](200)) val list4 = List(new Array[Byte](200), new Array[Byte](200)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER) store.put("list3", list3.iterator, StorageLevel.DISK_ONLY) Thread.sleep(100) // At this point LRU should not kick in because list3 is only on disk @@ -238,7 +319,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list3") != None, "list1 was not in store") assert(store.get("list3").get.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.DISK_AND_MEMORY) + store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER) Thread.sleep(100) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2") != None, "list3 was not in store") @@ -260,4 +341,13 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(stream.read() === -1, "end of stream not signalled") assert(stream.read(temp, 0, temp.length) === -1, "end of stream not signalled") } + + test("overly large block") { + val store = new BlockManager(master, new KryoSerializer, 500) + store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) + assert(store.getSingle("a1") === None, "a1 was in store") + store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) + assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") + assert(store.getSingle("a2") != None, "a2 was not in store") + } } |