Merge branch 'master' of github.com:mesos/spark into blockmanager
This commit is contained in:
commit
93091f6936
|
@ -107,7 +107,7 @@ abstract class RDD[T: ClassManifest](
|
||||||
// =======================================================================
|
// =======================================================================
|
||||||
|
|
||||||
/** A unique ID for this RDD (within its SparkContext). */
|
/** A unique ID for this RDD (within its SparkContext). */
|
||||||
val id = sc.newRddId()
|
val id: Int = sc.newRddId()
|
||||||
|
|
||||||
/** A friendly name for this RDD */
|
/** A friendly name for this RDD */
|
||||||
var name: String = null
|
var name: String = null
|
||||||
|
@ -120,7 +120,8 @@ abstract class RDD[T: ClassManifest](
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set this RDD's storage level to persist its values across operations after the first time
|
* Set this RDD's storage level to persist its values across operations after the first time
|
||||||
* it is computed. Can only be called once on each RDD.
|
* it is computed. This can only be used to assign a new storage level if the RDD does not
|
||||||
|
* have a storage level set yet..
|
||||||
*/
|
*/
|
||||||
def persist(newLevel: StorageLevel): RDD[T] = {
|
def persist(newLevel: StorageLevel): RDD[T] = {
|
||||||
// TODO: Handle changes of StorageLevel
|
// TODO: Handle changes of StorageLevel
|
||||||
|
@ -140,6 +141,15 @@ abstract class RDD[T: ClassManifest](
|
||||||
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
|
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
|
||||||
def cache(): RDD[T] = persist()
|
def cache(): RDD[T] = persist()
|
||||||
|
|
||||||
|
/** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */
|
||||||
|
def unpersist(): RDD[T] = {
|
||||||
|
logInfo("Removing RDD " + id + " from persistence list")
|
||||||
|
sc.env.blockManager.master.removeRdd(id)
|
||||||
|
sc.persistentRdds.remove(id)
|
||||||
|
storageLevel = StorageLevel.NONE
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
/** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
|
/** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
|
||||||
def getStorageLevel = storageLevel
|
def getStorageLevel = storageLevel
|
||||||
|
|
||||||
|
|
|
@ -1,47 +1,50 @@
|
||||||
package spark
|
package spark
|
||||||
|
|
||||||
import java.io._
|
import java.io._
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
|
||||||
import java.net.URI
|
import java.net.URI
|
||||||
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
import scala.collection.Map
|
import scala.collection.Map
|
||||||
import scala.collection.generic.Growable
|
import scala.collection.generic.Growable
|
||||||
import scala.collection.mutable.HashMap
|
import scala.collection.mutable.{ConcurrentMap, HashMap}
|
||||||
import scala.collection.JavaConversions._
|
|
||||||
|
import akka.actor.Actor._
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path
|
|
||||||
import org.apache.hadoop.conf.Configuration
|
import org.apache.hadoop.conf.Configuration
|
||||||
import org.apache.hadoop.mapred.InputFormat
|
import org.apache.hadoop.fs.Path
|
||||||
import org.apache.hadoop.mapred.SequenceFileInputFormat
|
import org.apache.hadoop.io.ArrayWritable
|
||||||
import org.apache.hadoop.io.Writable
|
|
||||||
import org.apache.hadoop.io.IntWritable
|
|
||||||
import org.apache.hadoop.io.LongWritable
|
|
||||||
import org.apache.hadoop.io.FloatWritable
|
|
||||||
import org.apache.hadoop.io.DoubleWritable
|
|
||||||
import org.apache.hadoop.io.BooleanWritable
|
import org.apache.hadoop.io.BooleanWritable
|
||||||
import org.apache.hadoop.io.BytesWritable
|
import org.apache.hadoop.io.BytesWritable
|
||||||
import org.apache.hadoop.io.ArrayWritable
|
import org.apache.hadoop.io.DoubleWritable
|
||||||
|
import org.apache.hadoop.io.FloatWritable
|
||||||
|
import org.apache.hadoop.io.IntWritable
|
||||||
|
import org.apache.hadoop.io.LongWritable
|
||||||
import org.apache.hadoop.io.NullWritable
|
import org.apache.hadoop.io.NullWritable
|
||||||
import org.apache.hadoop.io.Text
|
import org.apache.hadoop.io.Text
|
||||||
|
import org.apache.hadoop.io.Writable
|
||||||
import org.apache.hadoop.mapred.FileInputFormat
|
import org.apache.hadoop.mapred.FileInputFormat
|
||||||
|
import org.apache.hadoop.mapred.InputFormat
|
||||||
import org.apache.hadoop.mapred.JobConf
|
import org.apache.hadoop.mapred.JobConf
|
||||||
|
import org.apache.hadoop.mapred.SequenceFileInputFormat
|
||||||
import org.apache.hadoop.mapred.TextInputFormat
|
import org.apache.hadoop.mapred.TextInputFormat
|
||||||
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
|
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
|
||||||
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
|
|
||||||
import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
|
import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
|
||||||
|
|
||||||
import org.apache.mesos.MesosNativeLibrary
|
import org.apache.mesos.MesosNativeLibrary
|
||||||
|
|
||||||
import spark.deploy.{SparkHadoopUtil, LocalSparkCluster}
|
import spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
|
||||||
import spark.partial.ApproximateEvaluator
|
import spark.partial.{ApproximateEvaluator, PartialResult}
|
||||||
import spark.partial.PartialResult
|
|
||||||
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
|
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
|
||||||
import spark.scheduler._
|
import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler}
|
||||||
|
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler}
|
||||||
import spark.scheduler.local.LocalScheduler
|
import spark.scheduler.local.LocalScheduler
|
||||||
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
|
|
||||||
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
|
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
|
||||||
import spark.storage.BlockManagerUI
|
import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo}
|
||||||
import spark.util.{MetadataCleaner, TimeStampedHashMap}
|
import spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||||
import spark.storage.{StorageStatus, StorageUtils, RDDInfo}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
|
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
|
||||||
|
@ -97,7 +100,7 @@ class SparkContext(
|
||||||
private[spark] val addedJars = HashMap[String, Long]()
|
private[spark] val addedJars = HashMap[String, Long]()
|
||||||
|
|
||||||
// Keeps track of all persisted RDDs
|
// Keeps track of all persisted RDDs
|
||||||
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]()
|
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
|
||||||
private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
|
private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
|
||||||
|
|
||||||
|
|
||||||
|
@ -505,7 +508,7 @@ class SparkContext(
|
||||||
* Return information about what RDDs are cached, if they are in mem or on disk, how much space
|
* Return information about what RDDs are cached, if they are in mem or on disk, how much space
|
||||||
* they take, etc.
|
* they take, etc.
|
||||||
*/
|
*/
|
||||||
def getRDDStorageInfo : Array[RDDInfo] = {
|
def getRDDStorageInfo: Array[RDDInfo] = {
|
||||||
StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
|
StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -516,7 +519,7 @@ class SparkContext(
|
||||||
/**
|
/**
|
||||||
* Return information about blocks stored in all of the slaves
|
* Return information about blocks stored in all of the slaves
|
||||||
*/
|
*/
|
||||||
def getExecutorStorageStatus : Array[StorageStatus] = {
|
def getExecutorStorageStatus: Array[StorageStatus] = {
|
||||||
env.blockManager.master.getStorageStatus
|
env.blockManager.master.getStorageStatus
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -16,10 +16,16 @@ JavaRDDLike[T, JavaRDD[T]] {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set this RDD's storage level to persist its values across operations after the first time
|
* Set this RDD's storage level to persist its values across operations after the first time
|
||||||
* it is computed. Can only be called once on each RDD.
|
* it is computed. This can only be used to assign a new storage level if the RDD does not
|
||||||
|
* have a storage level set yet..
|
||||||
*/
|
*/
|
||||||
def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel))
|
def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
|
||||||
|
*/
|
||||||
|
def unpersist(): JavaRDD[T] = wrapRDD(rdd.unpersist())
|
||||||
|
|
||||||
// Transformations (return a new RDD)
|
// Transformations (return a new RDD)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -188,6 +188,38 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
|
||||||
} )
|
} )
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MUST be called within selector loop - else deadlock.
|
||||||
|
private def triggerForceCloseByException(key: SelectionKey, e: Exception) {
|
||||||
|
try {
|
||||||
|
key.interestOps(0)
|
||||||
|
} catch {
|
||||||
|
// ignore exceptions
|
||||||
|
case e: Exception => logDebug("Ignoring exception", e)
|
||||||
|
}
|
||||||
|
|
||||||
|
val conn = connectionsByKey.getOrElse(key, null)
|
||||||
|
if (conn == null) return
|
||||||
|
|
||||||
|
// Pushing to connect threadpool
|
||||||
|
handleConnectExecutor.execute(new Runnable {
|
||||||
|
override def run() {
|
||||||
|
try {
|
||||||
|
conn.callOnExceptionCallback(e)
|
||||||
|
} catch {
|
||||||
|
// ignore exceptions
|
||||||
|
case e: Exception => logDebug("Ignoring exception", e)
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
conn.close()
|
||||||
|
} catch {
|
||||||
|
// ignore exceptions
|
||||||
|
case e: Exception => logDebug("Ignoring exception", e)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
def run() {
|
def run() {
|
||||||
try {
|
try {
|
||||||
while(!selectorThread.isInterrupted) {
|
while(!selectorThread.isInterrupted) {
|
||||||
|
@ -200,29 +232,76 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
|
||||||
|
|
||||||
while(!keyInterestChangeRequests.isEmpty) {
|
while(!keyInterestChangeRequests.isEmpty) {
|
||||||
val (key, ops) = keyInterestChangeRequests.dequeue
|
val (key, ops) = keyInterestChangeRequests.dequeue
|
||||||
val connection = connectionsByKey.getOrElse(key, null)
|
|
||||||
if (connection != null) {
|
|
||||||
val lastOps = key.interestOps()
|
|
||||||
key.interestOps(ops)
|
|
||||||
|
|
||||||
// hot loop - prevent materialization of string if trace not enabled.
|
try {
|
||||||
if (isTraceEnabled()) {
|
if (key.isValid) {
|
||||||
def intToOpStr(op: Int): String = {
|
val connection = connectionsByKey.getOrElse(key, null)
|
||||||
val opStrs = ArrayBuffer[String]()
|
if (connection != null) {
|
||||||
if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ"
|
val lastOps = key.interestOps()
|
||||||
if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE"
|
key.interestOps(ops)
|
||||||
if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT"
|
|
||||||
if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT"
|
// hot loop - prevent materialization of string if trace not enabled.
|
||||||
if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " "
|
if (isTraceEnabled()) {
|
||||||
|
def intToOpStr(op: Int): String = {
|
||||||
|
val opStrs = ArrayBuffer[String]()
|
||||||
|
if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ"
|
||||||
|
if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE"
|
||||||
|
if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT"
|
||||||
|
if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT"
|
||||||
|
if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " "
|
||||||
|
}
|
||||||
|
|
||||||
|
logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() +
|
||||||
|
"] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]")
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() +
|
logInfo("Key not valid ? " + key)
|
||||||
"] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]")
|
throw new CancelledKeyException()
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: CancelledKeyException => {
|
||||||
|
logInfo("key already cancelled ? " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
|
}
|
||||||
|
case e: Exception => {
|
||||||
|
logError("Exception processing key " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
val selectedKeysCount = selector.select()
|
val selectedKeysCount =
|
||||||
|
try {
|
||||||
|
selector.select()
|
||||||
|
} catch {
|
||||||
|
// Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently.
|
||||||
|
case e: CancelledKeyException => {
|
||||||
|
// Some keys within the selectors list are invalid/closed. clear them.
|
||||||
|
val allKeys = selector.keys().iterator()
|
||||||
|
|
||||||
|
while (allKeys.hasNext()) {
|
||||||
|
val key = allKeys.next()
|
||||||
|
try {
|
||||||
|
if (! key.isValid) {
|
||||||
|
logInfo("Key not valid ? " + key)
|
||||||
|
throw new CancelledKeyException()
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: CancelledKeyException => {
|
||||||
|
logInfo("key already cancelled ? " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
|
}
|
||||||
|
case e: Exception => {
|
||||||
|
logError("Exception processing key " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
0
|
||||||
|
}
|
||||||
|
|
||||||
if (selectedKeysCount == 0) {
|
if (selectedKeysCount == 0) {
|
||||||
logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys")
|
logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys")
|
||||||
}
|
}
|
||||||
|
@ -231,22 +310,39 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
val selectedKeys = selector.selectedKeys().iterator()
|
if (0 != selectedKeysCount) {
|
||||||
while (selectedKeys.hasNext()) {
|
val selectedKeys = selector.selectedKeys().iterator()
|
||||||
val key = selectedKeys.next
|
while (selectedKeys.hasNext()) {
|
||||||
selectedKeys.remove()
|
val key = selectedKeys.next
|
||||||
if (key.isValid) {
|
selectedKeys.remove()
|
||||||
if (key.isAcceptable) {
|
try {
|
||||||
acceptConnection(key)
|
if (key.isValid) {
|
||||||
} else
|
if (key.isAcceptable) {
|
||||||
if (key.isConnectable) {
|
acceptConnection(key)
|
||||||
triggerConnect(key)
|
} else
|
||||||
} else
|
if (key.isConnectable) {
|
||||||
if (key.isReadable) {
|
triggerConnect(key)
|
||||||
triggerRead(key)
|
} else
|
||||||
} else
|
if (key.isReadable) {
|
||||||
if (key.isWritable) {
|
triggerRead(key)
|
||||||
triggerWrite(key)
|
} else
|
||||||
|
if (key.isWritable) {
|
||||||
|
triggerWrite(key)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
logInfo("Key not valid ? " + key)
|
||||||
|
throw new CancelledKeyException()
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
// weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException.
|
||||||
|
case e: CancelledKeyException => {
|
||||||
|
logInfo("key already cancelled ? " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
|
}
|
||||||
|
case e: Exception => {
|
||||||
|
logError("Exception processing key " + key, e)
|
||||||
|
triggerForceCloseByException(key, e)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,6 +15,7 @@ import akka.util.duration._
|
||||||
|
|
||||||
import spark.{Logging, SparkException, Utils}
|
import spark.{Logging, SparkException, Utils}
|
||||||
|
|
||||||
|
|
||||||
private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
|
private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
|
||||||
|
|
||||||
val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
|
val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
|
||||||
|
@ -87,6 +88,21 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
|
||||||
askDriverWithReply(RemoveBlock(blockId))
|
askDriverWithReply(RemoveBlock(blockId))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove all blocks belonging to the given RDD.
|
||||||
|
*/
|
||||||
|
def removeRdd(rddId: Int) {
|
||||||
|
val rddBlockPrefix = "rdd_" + rddId + "_"
|
||||||
|
// Get the list of blocks in block manager, and remove ones that are part of this RDD.
|
||||||
|
// The runtime complexity is linear to the number of blocks persisted in the cluster.
|
||||||
|
// It could be expensive if the cluster is large and has a lot of blocks persisted.
|
||||||
|
getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) =>
|
||||||
|
if (blockId.startsWith(rddBlockPrefix)) {
|
||||||
|
removeBlock(blockId)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the memory status for each block manager, in the form of a map from
|
* Return the memory status for each block manager, in the form of a map from
|
||||||
* the block manager's id to two long values. The first value is the maximum
|
* the block manager's id to two long values. The first value is the maximum
|
||||||
|
|
|
@ -236,8 +236,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
|
||||||
localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
|
localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
|
||||||
localDir = new File(rootDir, "spark-local-" + localDirId)
|
localDir = new File(rootDir, "spark-local-" + localDirId)
|
||||||
if (!localDir.exists) {
|
if (!localDir.exists) {
|
||||||
localDir.mkdirs()
|
foundLocalDir = localDir.mkdirs()
|
||||||
foundLocalDir = true
|
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case e: Exception =>
|
case e: Exception =>
|
||||||
|
|
|
@ -22,12 +22,17 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
|
||||||
}
|
}
|
||||||
|
|
||||||
case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
|
case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
|
||||||
numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) {
|
numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long)
|
||||||
|
extends Ordered[RDDInfo] {
|
||||||
override def toString = {
|
override def toString = {
|
||||||
import Utils.memoryBytesToString
|
import Utils.memoryBytesToString
|
||||||
"RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id,
|
"RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id,
|
||||||
storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize))
|
storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def compare(that: RDDInfo) = {
|
||||||
|
this.id - that.id
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Helper methods for storage-related objects */
|
/* Helper methods for storage-related objects */
|
||||||
|
@ -45,12 +50,12 @@ object StorageUtils {
|
||||||
sc: SparkContext) : Array[RDDInfo] = {
|
sc: SparkContext) : Array[RDDInfo] = {
|
||||||
|
|
||||||
// Group by rddId, ignore the partition name
|
// Group by rddId, ignore the partition name
|
||||||
val groupedRddBlocks = infos.groupBy { case(k, v) =>
|
val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) =>
|
||||||
k.substring(0,k.lastIndexOf('_'))
|
k.substring(0,k.lastIndexOf('_'))
|
||||||
}.mapValues(_.values.toArray)
|
}.mapValues(_.values.toArray)
|
||||||
|
|
||||||
// For each RDD, generate an RDDInfo object
|
// For each RDD, generate an RDDInfo object
|
||||||
groupedRddBlocks.map { case(rddKey, rddBlocks) =>
|
val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) =>
|
||||||
|
|
||||||
// Add up memory and disk sizes
|
// Add up memory and disk sizes
|
||||||
val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
|
val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
|
||||||
|
@ -65,6 +70,10 @@ object StorageUtils {
|
||||||
|
|
||||||
RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize)
|
RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize)
|
||||||
}.toArray
|
}.toArray
|
||||||
|
|
||||||
|
scala.util.Sorting.quickSort(rddInfos)
|
||||||
|
|
||||||
|
rddInfos
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Removes all BlockStatus object that are not part of a block prefix */
|
/* Removes all BlockStatus object that are not part of a block prefix */
|
||||||
|
|
|
@ -3,8 +3,10 @@ package spark
|
||||||
import network.ConnectionManagerId
|
import network.ConnectionManagerId
|
||||||
import org.scalatest.FunSuite
|
import org.scalatest.FunSuite
|
||||||
import org.scalatest.BeforeAndAfter
|
import org.scalatest.BeforeAndAfter
|
||||||
|
import org.scalatest.concurrent.Timeouts._
|
||||||
import org.scalatest.matchers.ShouldMatchers
|
import org.scalatest.matchers.ShouldMatchers
|
||||||
import org.scalatest.prop.Checkers
|
import org.scalatest.prop.Checkers
|
||||||
|
import org.scalatest.time.{Span, Millis}
|
||||||
import org.scalacheck.Arbitrary._
|
import org.scalacheck.Arbitrary._
|
||||||
import org.scalacheck.Gen
|
import org.scalacheck.Gen
|
||||||
import org.scalacheck.Prop._
|
import org.scalacheck.Prop._
|
||||||
|
@ -252,6 +254,30 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
|
||||||
assert(data2.count === 2)
|
assert(data2.count === 2)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("unpersist RDDs") {
|
||||||
|
DistributedSuite.amMaster = true
|
||||||
|
sc = new SparkContext("local-cluster[3,1,512]", "test")
|
||||||
|
val data = sc.parallelize(Seq(true, false, false, false), 4)
|
||||||
|
data.persist(StorageLevel.MEMORY_ONLY_2)
|
||||||
|
data.count
|
||||||
|
assert(sc.persistentRdds.isEmpty == false)
|
||||||
|
data.unpersist()
|
||||||
|
assert(sc.persistentRdds.isEmpty == true)
|
||||||
|
|
||||||
|
failAfter(Span(3000, Millis)) {
|
||||||
|
try {
|
||||||
|
while (! sc.getRDDStorageInfo.isEmpty) {
|
||||||
|
Thread.sleep(200)
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: Exception =>
|
||||||
|
// Do nothing. We might see exceptions because block manager
|
||||||
|
// is racing this thread to remove entries from the driver.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assert(sc.getRDDStorageInfo.isEmpty == true)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object DistributedSuite {
|
object DistributedSuite {
|
||||||
|
|
|
@ -2,6 +2,8 @@ package spark
|
||||||
|
|
||||||
import scala.collection.mutable.HashMap
|
import scala.collection.mutable.HashMap
|
||||||
import org.scalatest.FunSuite
|
import org.scalatest.FunSuite
|
||||||
|
import org.scalatest.concurrent.Timeouts._
|
||||||
|
import org.scalatest.time.{Span, Millis}
|
||||||
import spark.SparkContext._
|
import spark.SparkContext._
|
||||||
import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD}
|
import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD}
|
||||||
|
|
||||||
|
@ -100,6 +102,28 @@ class RDDSuite extends FunSuite with LocalSparkContext {
|
||||||
assert(rdd.collect().toList === List(1, 2, 3, 4))
|
assert(rdd.collect().toList === List(1, 2, 3, 4))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("unpersist RDD") {
|
||||||
|
sc = new SparkContext("local", "test")
|
||||||
|
val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache()
|
||||||
|
rdd.count
|
||||||
|
assert(sc.persistentRdds.isEmpty == false)
|
||||||
|
rdd.unpersist()
|
||||||
|
assert(sc.persistentRdds.isEmpty == true)
|
||||||
|
|
||||||
|
failAfter(Span(3000, Millis)) {
|
||||||
|
try {
|
||||||
|
while (! sc.getRDDStorageInfo.isEmpty) {
|
||||||
|
Thread.sleep(200)
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: Exception =>
|
||||||
|
// Do nothing. We might see exceptions because block manager
|
||||||
|
// is racing this thread to remove entries from the driver.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assert(sc.getRDDStorageInfo.isEmpty == true)
|
||||||
|
}
|
||||||
|
|
||||||
test("caching with failures") {
|
test("caching with failures") {
|
||||||
sc = new SparkContext("local", "test")
|
sc = new SparkContext("local", "test")
|
||||||
val onlySplit = new Partition { override def index: Int = 0 }
|
val onlySplit = new Partition { override def index: Int = 0 }
|
||||||
|
|
|
@ -207,6 +207,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("removing rdd") {
|
||||||
|
store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
|
||||||
|
val a1 = new Array[Byte](400)
|
||||||
|
val a2 = new Array[Byte](400)
|
||||||
|
val a3 = new Array[Byte](400)
|
||||||
|
// Putting a1, a2 and a3 in memory.
|
||||||
|
store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY)
|
||||||
|
store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY)
|
||||||
|
store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY)
|
||||||
|
master.removeRdd(0)
|
||||||
|
|
||||||
|
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
|
||||||
|
store.getSingle("rdd_0_0") should be (None)
|
||||||
|
master.getLocations("rdd_0_0") should have size 0
|
||||||
|
}
|
||||||
|
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
|
||||||
|
store.getSingle("rdd_0_1") should be (None)
|
||||||
|
master.getLocations("rdd_0_1") should have size 0
|
||||||
|
}
|
||||||
|
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
|
||||||
|
store.getSingle("nonrddblock") should not be (None)
|
||||||
|
master.getLocations("nonrddblock") should have size (1)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("reregistration on heart beat") {
|
test("reregistration on heart beat") {
|
||||||
val heartBeat = PrivateMethod[Unit]('heartBeat)
|
val heartBeat = PrivateMethod[Unit]('heartBeat)
|
||||||
store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
|
store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
|
||||||
|
|
|
@ -29,7 +29,7 @@ object SparkBuild extends Build {
|
||||||
|
|
||||||
lazy val core = Project("core", file("core"), settings = coreSettings)
|
lazy val core = Project("core", file("core"), settings = coreSettings)
|
||||||
|
|
||||||
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming)
|
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core)
|
||||||
|
|
||||||
lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
|
lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
|
||||||
|
|
||||||
|
|
14
repl/pom.xml
14
repl/pom.xml
|
@ -96,13 +96,6 @@
|
||||||
<classifier>hadoop1</classifier>
|
<classifier>hadoop1</classifier>
|
||||||
<scope>runtime</scope>
|
<scope>runtime</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
|
||||||
<groupId>org.spark-project</groupId>
|
|
||||||
<artifactId>spark-streaming</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
<classifier>hadoop1</classifier>
|
|
||||||
<scope>runtime</scope>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-core</artifactId>
|
<artifactId>hadoop-core</artifactId>
|
||||||
|
@ -147,13 +140,6 @@
|
||||||
<classifier>hadoop2</classifier>
|
<classifier>hadoop2</classifier>
|
||||||
<scope>runtime</scope>
|
<scope>runtime</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
|
||||||
<groupId>org.spark-project</groupId>
|
|
||||||
<artifactId>spark-streaming</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
<classifier>hadoop2</classifier>
|
|
||||||
<scope>runtime</scope>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-core</artifactId>
|
<artifactId>hadoop-core</artifactId>
|
||||||
|
|
7
run
7
run
|
@ -95,6 +95,7 @@ export JAVA_OPTS
|
||||||
|
|
||||||
CORE_DIR="$FWDIR/core"
|
CORE_DIR="$FWDIR/core"
|
||||||
REPL_DIR="$FWDIR/repl"
|
REPL_DIR="$FWDIR/repl"
|
||||||
|
REPL_BIN_DIR="$FWDIR/repl-bin"
|
||||||
EXAMPLES_DIR="$FWDIR/examples"
|
EXAMPLES_DIR="$FWDIR/examples"
|
||||||
BAGEL_DIR="$FWDIR/bagel"
|
BAGEL_DIR="$FWDIR/bagel"
|
||||||
STREAMING_DIR="$FWDIR/streaming"
|
STREAMING_DIR="$FWDIR/streaming"
|
||||||
|
@ -125,8 +126,8 @@ if [ -e "$FWDIR/lib_managed" ]; then
|
||||||
CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
|
CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
|
||||||
fi
|
fi
|
||||||
CLASSPATH+=":$REPL_DIR/lib/*"
|
CLASSPATH+=":$REPL_DIR/lib/*"
|
||||||
if [ -e repl-bin/target ]; then
|
if [ -e $REPL_BIN_DIR/target ]; then
|
||||||
for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
|
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
|
||||||
CLASSPATH+=":$jar"
|
CLASSPATH+=":$jar"
|
||||||
done
|
done
|
||||||
fi
|
fi
|
||||||
|
@ -134,7 +135,6 @@ CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
|
||||||
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
|
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
|
||||||
CLASSPATH+=":$jar"
|
CLASSPATH+=":$jar"
|
||||||
done
|
done
|
||||||
export CLASSPATH # Needed for spark-shell
|
|
||||||
|
|
||||||
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
|
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
|
||||||
# to avoid the -sources and -doc packages that are built by publish-local.
|
# to avoid the -sources and -doc packages that are built by publish-local.
|
||||||
|
@ -163,4 +163,5 @@ else
|
||||||
EXTRA_ARGS="$JAVA_OPTS"
|
EXTRA_ARGS="$JAVA_OPTS"
|
||||||
fi
|
fi
|
||||||
|
|
||||||
|
export CLASSPATH # Needed for spark-shell
|
||||||
exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@"
|
exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@"
|
||||||
|
|
Loading…
Reference in a new issue