Merge branch 'master' into graphx

This commit is contained in:
Reynold Xin 2014-01-13 16:21:26 -08:00
commit e2d25d2dfe
233 changed files with 5023 additions and 1441 deletions

View file

@ -38,7 +38,6 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
}
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}
test("halting by voting") {

View file

@ -59,12 +59,7 @@ if [ -n "$IPYTHON_OPTS" ]; then
fi
if [[ "$IPYTHON" = "1" ]] ; then
# IPython <1.0.0 doesn't honor PYTHONSTARTUP, while 1.0.0+ does.
# Hence we clear PYTHONSTARTUP and use the -c "%run $IPYTHONSTARTUP" command which works on all versions
# We also force interactive mode with "-i"
IPYTHONSTARTUP=$PYTHONSTARTUP
PYTHONSTARTUP=
exec ipython "$IPYTHON_OPTS" -i -c "%run $IPYTHONSTARTUP"
exec ipython $IPYTHON_OPTS
else
exec "$PYSPARK_PYTHON" "$@"
fi

View file

@ -1,8 +1,11 @@
# Set everything to be logged to the console
log4j.rootCategory=INFO, console
log4j.appender.console=org.apache.log4j.ConsoleAppender
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
# Ignore messages below warning level from Jetty, because it's a bit verbose
# Settings to quiet third party logs that are too verbose
log4j.logger.org.eclipse.jetty=WARN
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO

View file

@ -98,6 +98,11 @@
<groupId>${akka.group}</groupId>
<artifactId>akka-slf4j_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
<artifactId>akka-testkit_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
@ -165,6 +170,11 @@
<artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>

View file

@ -1,8 +1,11 @@
# Set everything to be logged to the console
log4j.rootCategory=INFO, console
log4j.appender.console=org.apache.log4j.ConsoleAppender
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
# Ignore messages below warning level from Jetty, because it's a bit verbose
# Settings to quiet third party logs that are too verbose
log4j.logger.org.eclipse.jetty=WARN
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO

View file

@ -218,7 +218,7 @@ private object Accumulators {
def newId: Long = synchronized {
lastId += 1
return lastId
lastId
}
def register(a: Accumulable[_, _], original: Boolean): Unit = synchronized {

View file

@ -17,7 +17,7 @@
package org.apache.spark
import org.apache.spark.util.AppendOnlyMap
import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap}
/**
* A set of functions used to aggregate data.
@ -31,30 +31,51 @@ case class Aggregator[K, V, C] (
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C) {
private val sparkConf = SparkEnv.get.conf
private val externalSorting = sparkConf.getBoolean("spark.shuffle.externalSorting", true)
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = {
val combiners = new AppendOnlyMap[K, C]
var kv: Product2[K, V] = null
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
var kv: Product2[K, V] = null
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
}
while (iter.hasNext) {
kv = iter.next()
combiners.changeValue(kv._1, update)
}
combiners.iterator
} else {
val combiners =
new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners)
while (iter.hasNext) {
val (k, v) = iter.next()
combiners.insert(k, v)
}
combiners.iterator
}
while (iter.hasNext) {
kv = iter.next()
combiners.changeValue(kv._1, update)
}
combiners.iterator
}
def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
val combiners = new AppendOnlyMap[K, C]
var kc: (K, C) = null
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
var kc: Product2[K, C] = null
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2
}
while (iter.hasNext) {
kc = iter.next()
combiners.changeValue(kc._1, update)
}
combiners.iterator
} else {
val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners)
while (iter.hasNext) {
val (k, c) = iter.next()
combiners.insert(k, c)
}
combiners.iterator
}
while (iter.hasNext) {
kc = iter.next()
combiners.changeValue(kc._1, update)
}
combiners.iterator
}
}

View file

@ -38,7 +38,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
blockManager.get(key) match {
case Some(values) =>
// Partition is already materialized, so just return its values
return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
case None =>
// Mark the split as loading (unless someone else marks it first)
@ -74,7 +74,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
val elements = new ArrayBuffer[Any]
elements ++= computedValues
blockManager.put(key, elements, storageLevel, tellMaster = true)
return elements.iterator.asInstanceOf[Iterator[T]]
elements.iterator.asInstanceOf[Iterator[T]]
} finally {
loading.synchronized {
loading.remove(key)

View file

@ -47,17 +47,17 @@ private[spark] class HttpFileServer extends Logging {
def addFile(file: File) : String = {
addFileToDir(file, fileDir)
return serverUri + "/files/" + file.getName
serverUri + "/files/" + file.getName
}
def addJar(file: File) : String = {
addFileToDir(file, jarDir)
return serverUri + "/jars/" + file.getName
serverUri + "/jars/" + file.getName
}
def addFileToDir(file: File, dir: File) : String = {
Files.copy(file, new File(dir, file.getName))
return dir + "/" + file.getName
dir + "/" + file.getName
}
}

View file

@ -41,7 +41,7 @@ trait Logging {
}
log_ = LoggerFactory.getLogger(className)
}
return log_
log_
}
// Log methods that take only a String

View file

@ -32,15 +32,16 @@ import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
private[spark] sealed trait MapOutputTrackerMessage
private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String)
private[spark] case class GetMapOutputStatuses(shuffleId: Int)
extends MapOutputTrackerMessage
private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage
private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster)
extends Actor with Logging {
def receive = {
case GetMapOutputStatuses(shuffleId: Int, requester: String) =>
logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + requester)
case GetMapOutputStatuses(shuffleId: Int) =>
val hostPort = sender.path.address.hostPort
logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort)
sender ! tracker.getSerializedMapOutputStatuses(shuffleId)
case StopMapOutputTracker =>
@ -119,11 +120,10 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging {
if (fetchedStatuses == null) {
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
val hostPort = Utils.localHostPort(conf)
// This try-finally prevents hangs due to timeouts:
try {
val fetchedBytes =
askTracker(GetMapOutputStatuses(shuffleId, hostPort)).asInstanceOf[Array[Byte]]
askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]]
fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes)
logInfo("Got the output locations")
mapStatuses.put(shuffleId, fetchedStatuses)
@ -139,7 +139,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging {
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses)
}
}
else{
else {
throw new FetchFailedException(null, shuffleId, -1, reduceId,
new Exception("Missing all output locations for shuffle " + shuffleId))
}

View file

@ -53,15 +53,16 @@ object Partitioner {
return r.partitioner.get
}
if (rdd.context.conf.contains("spark.default.parallelism")) {
return new HashPartitioner(rdd.context.defaultParallelism)
new HashPartitioner(rdd.context.defaultParallelism)
} else {
return new HashPartitioner(bySize.head.partitions.size)
new HashPartitioner(bySize.head.partitions.size)
}
}
}
/**
* A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
* A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using
* Java's `Object.hashCode`.
*
* Java arrays have hashCodes that are based on the arrays' identities rather than their contents,
* so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will
@ -84,8 +85,8 @@ class HashPartitioner(partitions: Int) extends Partitioner {
}
/**
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
* Determines the ranges by sampling the RDD passed in.
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly
* equal ranges. The ranges are determined by sampling the content of the RDD passed in.
*/
class RangePartitioner[K <% Ordered[K]: ClassTag, V](
partitions: Int,

View file

@ -31,9 +31,9 @@ import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable,
FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable}
FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable}
import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat,
TextInputFormat}
TextInputFormat}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob}
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.mesos.MesosNativeLibrary
@ -49,7 +49,7 @@ import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType,
ClosureCleaner}
ClosureCleaner}
/**
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@ -116,7 +116,7 @@ class SparkContext(
throw new SparkException("An application must be set in your configuration")
}
if (conf.get("spark.logConf", "false").toBoolean) {
if (conf.getBoolean("spark.logConf", false)) {
logInfo("Spark configuration:\n" + conf.toDebugString)
}
@ -244,6 +244,10 @@ class SparkContext(
localProperties.set(new Properties())
}
/**
* Set a local property that affects jobs submitted from this thread, such as the
* Spark fair scheduler pool.
*/
def setLocalProperty(key: String, value: String) {
if (localProperties.get() == null) {
localProperties.set(new Properties())
@ -255,6 +259,10 @@ class SparkContext(
}
}
/**
* Get a local property set in this thread, or null if it is missing. See
* [[org.apache.spark.SparkContext.setLocalProperty]].
*/
def getLocalProperty(key: String): String =
Option(localProperties.get).map(_.getProperty(key)).getOrElse(null)
@ -265,7 +273,7 @@ class SparkContext(
}
/**
* Assigns a group id to all the jobs started by this thread until the group id is set to a
* Assigns a group ID to all the jobs started by this thread until the group ID is set to a
* different value or cleared.
*
* Often, a unit of execution in an application consists of multiple Spark actions or jobs.
@ -288,7 +296,7 @@ class SparkContext(
setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId)
}
/** Clear the job group id and its description. */
/** Clear the current thread's job group ID and its description. */
def clearJobGroup() {
setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null)
setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null)
@ -337,29 +345,42 @@ class SparkContext(
}
/**
* Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and any
* other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
* etc).
* Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other
* necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable),
* using the older MapReduce API (`org.apache.hadoop.mapred`).
*
* @param conf JobConf for setting up the dataset
* @param inputFormatClass Class of the [[InputFormat]]
* @param keyClass Class of the keys
* @param valueClass Class of the values
* @param minSplits Minimum number of Hadoop Splits to generate.
* @param cloneRecords If true, Spark will clone the records produced by Hadoop RecordReader.
* Most RecordReader implementations reuse wrapper objects across multiple
* records, and can cause problems in RDD collect or aggregation operations.
* By default the records are cloned in Spark. However, application
* programmers can explicitly disable the cloning for better performance.
*/
def hadoopRDD[K, V](
def hadoopRDD[K: ClassTag, V: ClassTag](
conf: JobConf,
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int = defaultMinSplits
minSplits: Int = defaultMinSplits,
cloneRecords: Boolean = true
): RDD[(K, V)] = {
// Add necessary security credentials to the JobConf before broadcasting it.
SparkHadoopUtil.get.addCredentials(conf)
new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits, cloneRecords)
}
/** Get an RDD for a Hadoop file with an arbitrary InputFormat */
def hadoopFile[K, V](
def hadoopFile[K: ClassTag, V: ClassTag](
path: String,
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int = defaultMinSplits
minSplits: Int = defaultMinSplits,
cloneRecords: Boolean = true
): RDD[(K, V)] = {
// A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it.
val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration))
@ -371,7 +392,8 @@ class SparkContext(
inputFormatClass,
keyClass,
valueClass,
minSplits)
minSplits,
cloneRecords)
}
/**
@ -382,14 +404,15 @@ class SparkContext(
* val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits)
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int)
(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F])
: RDD[(K, V)] = {
def hadoopFile[K, V, F <: InputFormat[K, V]]
(path: String, minSplits: Int, cloneRecords: Boolean = true)
(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
hadoopFile(path,
fm.runtimeClass.asInstanceOf[Class[F]],
km.runtimeClass.asInstanceOf[Class[K]],
vm.runtimeClass.asInstanceOf[Class[V]],
minSplits)
fm.runtimeClass.asInstanceOf[Class[F]],
km.runtimeClass.asInstanceOf[Class[K]],
vm.runtimeClass.asInstanceOf[Class[V]],
minSplits,
cloneRecords)
}
/**
@ -400,61 +423,67 @@ class SparkContext(
* val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path)
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, cloneRecords: Boolean = true)
(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] =
hadoopFile[K, V, F](path, defaultMinSplits)
hadoopFile[K, V, F](path, defaultMinSplits, cloneRecords)
/** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String)
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]
(path: String, cloneRecords: Boolean = true)
(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
newAPIHadoopFile(
path,
fm.runtimeClass.asInstanceOf[Class[F]],
km.runtimeClass.asInstanceOf[Class[K]],
vm.runtimeClass.asInstanceOf[Class[V]])
path,
fm.runtimeClass.asInstanceOf[Class[F]],
km.runtimeClass.asInstanceOf[Class[K]],
vm.runtimeClass.asInstanceOf[Class[V]],
cloneRecords = cloneRecords)
}
/**
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*/
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](
def newAPIHadoopFile[K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]](
path: String,
fClass: Class[F],
kClass: Class[K],
vClass: Class[V],
conf: Configuration = hadoopConfiguration): RDD[(K, V)] = {
conf: Configuration = hadoopConfiguration,
cloneRecords: Boolean = true): RDD[(K, V)] = {
val job = new NewHadoopJob(conf)
NewFileInputFormat.addInputPath(job, new Path(path))
val updatedConf = job.getConfiguration
new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf)
new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf, cloneRecords)
}
/**
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*/
def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]](
def newAPIHadoopRDD[K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]](
conf: Configuration = hadoopConfiguration,
fClass: Class[F],
kClass: Class[K],
vClass: Class[V]): RDD[(K, V)] = {
new NewHadoopRDD(this, fClass, kClass, vClass, conf)
vClass: Class[V],
cloneRecords: Boolean = true): RDD[(K, V)] = {
new NewHadoopRDD(this, fClass, kClass, vClass, conf, cloneRecords)
}
/** Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String,
def sequenceFile[K: ClassTag, V: ClassTag](path: String,
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int
minSplits: Int,
cloneRecords: Boolean = true
): RDD[(K, V)] = {
val inputFormatClass = classOf[SequenceFileInputFormat[K, V]]
hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)
hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits, cloneRecords)
}
/** Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] =
sequenceFile(path, keyClass, valueClass, defaultMinSplits)
def sequenceFile[K: ClassTag, V: ClassTag](path: String, keyClass: Class[K], valueClass: Class[V],
cloneRecords: Boolean = true): RDD[(K, V)] =
sequenceFile(path, keyClass, valueClass, defaultMinSplits, cloneRecords)
/**
* Version of sequenceFile() for types implicitly convertible to Writables through a
@ -472,17 +501,18 @@ class SparkContext(
* for the appropriate type. In addition, we pass the converter a ClassTag of its type to
* allow it to figure out the Writable class to use in the subclass case.
*/
def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits)
(implicit km: ClassTag[K], vm: ClassTag[V],
kcf: () => WritableConverter[K], vcf: () => WritableConverter[V])
def sequenceFile[K, V]
(path: String, minSplits: Int = defaultMinSplits, cloneRecords: Boolean = true)
(implicit km: ClassTag[K], vm: ClassTag[V],
kcf: () => WritableConverter[K], vcf: () => WritableConverter[V])
: RDD[(K, V)] = {
val kc = kcf()
val vc = vcf()
val format = classOf[SequenceFileInputFormat[Writable, Writable]]
val writables = hadoopFile(path, format,
kc.writableClass(km).asInstanceOf[Class[Writable]],
vc.writableClass(vm).asInstanceOf[Class[Writable]], minSplits)
writables.map{case (k,v) => (kc.convert(k), vc.convert(v))}
vc.writableClass(vm).asInstanceOf[Class[Writable]], minSplits, cloneRecords)
writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) }
}
/**
@ -517,15 +547,15 @@ class SparkContext(
// Methods for creating shared variables
/**
* Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values
* to using the `+=` method. Only the driver can access the accumulator's `value`.
* Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add"
* values to using the `+=` method. Only the driver can access the accumulator's `value`.
*/
def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
new Accumulator(initialValue, param)
/**
* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
* Only the driver can access the accumuable's `value`.
* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values
* with `+=`. Only the driver can access the accumuable's `value`.
* @tparam T accumulator type
* @tparam R type that can be added to the accumulator
*/
@ -538,14 +568,16 @@ class SparkContext(
* 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) = {
def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T]
(initialValue: R) = {
val param = new GrowableAccumulableParam[R,T]
new Accumulable(initialValue, param)
}
/**
* Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.broadcast.Broadcast]] object for
* reading it in distributed functions. The variable will be sent to each cluster only once.
* Broadcast a read-only variable to the cluster, returning a
* [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions.
* The variable will be sent to each cluster only once.
*/
def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
@ -667,10 +699,10 @@ class SparkContext(
key = uri.getScheme match {
// A JAR file which exists only on the driver node
case null | "file" =>
if (SparkHadoopUtil.get.isYarnMode()) {
// In order for this to work on yarn the user must specify the --addjars option to
// the client to upload the file into the distributed cache to make it show up in the
// current working directory.
if (SparkHadoopUtil.get.isYarnMode() && master == "yarn-standalone") {
// In order for this to work in yarn standalone mode the user must specify the
// --addjars option to the client to upload the file into the distributed cache
// of the AM to make it show up in the current working directory.
val fileName = new Path(uri.getPath).getName()
try {
env.httpFileServer.addJar(new File(fileName))
@ -754,8 +786,11 @@ class SparkContext(
private[spark] def getCallSite(): String = {
val callSite = getLocalProperty("externalCallSite")
if (callSite == null) return Utils.formatSparkCallSite
callSite
if (callSite == null) {
Utils.formatSparkCallSite
} else {
callSite
}
}
/**
@ -905,7 +940,7 @@ class SparkContext(
*/
private[spark] def clean[F <: AnyRef](f: F): F = {
ClosureCleaner.clean(f)
return f
f
}
/**
@ -917,7 +952,7 @@ class SparkContext(
val path = new Path(dir, UUID.randomUUID().toString)
val fs = path.getFileSystem(hadoopConfiguration)
fs.mkdirs(path)
fs.getFileStatus(path).getPath().toString
fs.getFileStatus(path).getPath.toString
}
}
@ -1010,7 +1045,8 @@ object SparkContext {
implicit def stringToText(s: String) = new Text(s)
private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = {
private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T])
: ArrayWritable = {
def anyToWritable[U <% Writable](u: U): Writable = u
new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]],
@ -1033,7 +1069,9 @@ object SparkContext {
implicit def booleanWritableConverter() = simpleWritableConverter[Boolean, BooleanWritable](_.get)
implicit def bytesWritableConverter() = simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes)
implicit def bytesWritableConverter() = {
simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes)
}
implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString)
@ -1049,7 +1087,8 @@ object SparkContext {
if (uri != null) {
val uriStr = uri.toString
if (uriStr.startsWith("jar:file:")) {
// URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", so pull out the /path/foo.jar
// URI will be of the form "jar:file:/path/foo.jar!/package/cls.class",
// so pull out the /path/foo.jar
List(uriStr.substring("jar:file:".length, uriStr.indexOf('!')))
} else {
Nil
@ -1072,7 +1111,7 @@ object SparkContext {
* parameters that are passed as the default value of null, instead of throwing an exception
* like SparkConf would.
*/
private def updatedConf(
private[spark] def updatedConf(
conf: SparkConf,
master: String,
appName: String,
@ -1203,7 +1242,7 @@ object SparkContext {
case mesosUrl @ MESOS_REGEX(_) =>
MesosNativeLibrary.load()
val scheduler = new TaskSchedulerImpl(sc)
val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean
val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false)
val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs
val backend = if (coarseGrained) {
new CoarseMesosSchedulerBackend(scheduler, sc, url, appName)

View file

@ -54,7 +54,11 @@ class SparkEnv private[spark] (
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
val metricsSystem: MetricsSystem,
val conf: SparkConf) {
val conf: SparkConf) extends Logging {
// A mapping of thread ID to amount of memory used for shuffle in bytes
// All accesses should be manually synchronized
val shuffleMemoryMap = mutable.HashMap[Long, Long]()
private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
@ -128,16 +132,6 @@ object SparkEnv extends Logging {
conf.set("spark.driver.port", boundPort.toString)
}
// set only if unset until now.
if (!conf.contains("spark.hostPort")) {
if (!isDriver){
// unexpected
Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set")
}
Utils.checkHost(hostname)
conf.set("spark.hostPort", hostname + ":" + boundPort)
}
val classLoader = Thread.currentThread.getContextClassLoader
// Create an instance of the class named by the given Java system property, or by
@ -162,7 +156,7 @@ object SparkEnv extends Logging {
actorSystem.actorOf(Props(newActor), name = name)
} else {
val driverHost: String = conf.get("spark.driver.host", "localhost")
val driverPort: Int = conf.get("spark.driver.port", "7077").toInt
val driverPort: Int = conf.getInt("spark.driver.port", 7077)
Utils.checkHost(driverHost, "Expected hostname")
val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name"
val timeout = AkkaUtils.lookupTimeout(conf)

View file

@ -134,28 +134,28 @@ class SparkHadoopWriter(@transient jobConf: JobConf)
format = conf.value.getOutputFormat()
.asInstanceOf[OutputFormat[AnyRef,AnyRef]]
}
return format
format
}
private def getOutputCommitter(): OutputCommitter = {
if (committer == null) {
committer = conf.value.getOutputCommitter
}
return committer
committer
}
private def getJobContext(): JobContext = {
if (jobContext == null) {
jobContext = newJobContext(conf.value, jID.value)
}
return jobContext
jobContext
}
private def getTaskContext(): TaskAttemptContext = {
if (taskContext == null) {
taskContext = newTaskAttemptContext(conf.value, taID.value)
}
return taskContext
taskContext
}
private def setIDs(jobid: Int, splitid: Int, attemptid: Int) {
@ -182,19 +182,18 @@ object SparkHadoopWriter {
def createJobID(time: Date, id: Int): JobID = {
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
val jobtrackerID = formatter.format(new Date())
return new JobID(jobtrackerID, id)
new JobID(jobtrackerID, id)
}
def createPathFromString(path: String, conf: JobConf): Path = {
if (path == null) {
throw new IllegalArgumentException("Output path is null")
}
var outputPath = new Path(path)
val outputPath = new Path(path)
val fs = outputPath.getFileSystem(conf)
if (outputPath == null || fs == null) {
throw new IllegalArgumentException("Incorrectly formatted output path")
}
outputPath = outputPath.makeQualified(fs)
return outputPath
outputPath.makeQualified(fs)
}
}

View file

@ -207,13 +207,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
* e.g. for the array
* [1,10,20,50] the buckets are [1,10) [10,20) [20,50]
* e.g 1<=x<10 , 10<=x<20, 20<=x<50
* And on the input of 1 and 50 we would have a histogram of 1,0,0
*
* And on the input of 1 and 50 we would have a histogram of 1,0,0
*
* Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
* from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
* to true.
* buckets must be sorted and not contain any duplicates.
* buckets array must be at least two elements
* buckets array must be at least two elements
* All NaN entries are treated the same. If you have a NaN bucket it must be
* the maximum value of the last position and all NaN entries will be counted
* in that bucket.
@ -225,6 +225,12 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = {
srdd.histogram(buckets.map(_.toDouble), evenBuckets)
}
/** Assign a name to this RDD */
def setName(name: String): JavaDoubleRDD = {
srdd.setName(name)
this
}
}
object JavaDoubleRDD {

View file

@ -647,6 +647,12 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K
def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = {
rdd.countApproxDistinctByKey(relativeSD, numPartitions)
}
/** Assign a name to this RDD */
def setName(name: String): JavaPairRDD[K, V] = {
rdd.setName(name)
this
}
}
object JavaPairRDD {

View file

@ -127,6 +127,12 @@ JavaRDDLike[T, JavaRDD[T]] {
wrapRDD(rdd.subtract(other, p))
override def toString = rdd.toString
/** Assign a name to this RDD */
def setName(name: String): JavaRDD[T] = {
rdd.setName(name)
this
}
}
object JavaRDD {

View file

@ -244,6 +244,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
new java.util.ArrayList(arr)
}
/**
* Return an array that contains all of the elements in this RDD.
*/
def toArray(): JList[T] = collect()
/**
* Return an array that contains all of the elements in a specific partition of this RDD.
*/
@ -455,4 +460,5 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
*/
def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD)
def name(): String = rdd.name
}

View file

@ -425,6 +425,51 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
def clearCallSite() {
sc.clearCallSite()
}
/**
* Set a local property that affects jobs submitted from this thread, such as the
* Spark fair scheduler pool.
*/
def setLocalProperty(key: String, value: String): Unit = sc.setLocalProperty(key, value)
/**
* Get a local property set in this thread, or null if it is missing. See
* [[org.apache.spark.api.java.JavaSparkContext.setLocalProperty]].
*/
def getLocalProperty(key: String): String = sc.getLocalProperty(key)
/**
* Assigns a group ID to all the jobs started by this thread until the group ID is set to a
* different value or cleared.
*
* Often, a unit of execution in an application consists of multiple Spark actions or jobs.
* Application programmers can use this method to group all those jobs together and give a
* group description. Once set, the Spark web UI will associate such jobs with this group.
*
* The application can also use [[org.apache.spark.api.java.JavaSparkContext.cancelJobGroup]]
* to cancel all running jobs in this group. For example,
* {{{
* // In the main thread:
* sc.setJobGroup("some_job_to_cancel", "some job description");
* rdd.map(...).count();
*
* // In a separate thread:
* sc.cancelJobGroup("some_job_to_cancel");
* }}}
*/
def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description)
/** Clear the current thread's job group ID and its description. */
def clearJobGroup(): Unit = sc.clearJobGroup()
/**
* Cancel active jobs for the specified group. See
* [[org.apache.spark.api.java.JavaSparkContext.setJobGroup]] for more information.
*/
def cancelJobGroup(groupId: String): Unit = sc.cancelJobGroup(groupId)
/** Cancel all jobs that have been scheduled or are running. */
def cancelAllJobs(): Unit = sc.cancelAllJobs()
}
object JavaSparkContext {
@ -436,5 +481,12 @@ object JavaSparkContext {
* Find the JAR from which a given class was loaded, to make it easy for users to pass
* their JARs to SparkContext.
*/
def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray
/**
* Find the JAR that contains the class of a particular object, to make it easy for users
* to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in
* your driver program.
*/
def jarOfObject(obj: AnyRef): Array[String] = SparkContext.jarOfObject(obj).toArray
}

View file

@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag](
accumulator: Accumulator[JList[Array[Byte]]])
extends RDD[Array[Byte]](parent) {
val bufferSize = conf.get("spark.buffer.size", "65536").toInt
val bufferSize = conf.getInt("spark.buffer.size", 65536)
override def getPartitions = parent.partitions
@ -95,7 +95,7 @@ private[spark] class PythonRDD[T: ClassTag](
// Return an iterator that read lines from the process's stdout
val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize))
return new Iterator[Array[Byte]] {
val stdoutIterator = new Iterator[Array[Byte]] {
def next(): Array[Byte] = {
val obj = _nextObj
if (hasNext) {
@ -156,6 +156,7 @@ private[spark] class PythonRDD[T: ClassTag](
def hasNext = _nextObj.length != 0
}
stdoutIterator
}
val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
@ -250,7 +251,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
Utils.checkHost(serverHost, "Expected hostname")
val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt
val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536)
override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList

View file

@ -92,8 +92,8 @@ private object HttpBroadcast extends Logging {
def initialize(isDriver: Boolean, conf: SparkConf) {
synchronized {
if (!initialized) {
bufferSize = conf.get("spark.buffer.size", "65536").toInt
compress = conf.get("spark.broadcast.compress", "true").toBoolean
bufferSize = conf.getInt("spark.buffer.size", 65536)
compress = conf.getBoolean("spark.broadcast.compress", true)
if (isDriver) {
createServer(conf)
conf.set("spark.httpBroadcast.uri", serverUri)

View file

@ -180,7 +180,7 @@ extends Logging {
initialized = false
}
lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024
lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024
def blockifyObject[T](obj: T): TorrentInfo = {
val byteArray = Utils.serialize[T](obj)
@ -203,16 +203,16 @@ extends Logging {
}
bais.close()
var tInfo = TorrentInfo(retVal, blockNum, byteArray.length)
val tInfo = TorrentInfo(retVal, blockNum, byteArray.length)
tInfo.hasBlocks = blockNum
return tInfo
tInfo
}
def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock],
totalBytes: Int,
totalBlocks: Int): T = {
var retByteArray = new Array[Byte](totalBytes)
val retByteArray = new Array[Byte](totalBytes)
for (i <- 0 until totalBlocks) {
System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray,
i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length)

View file

@ -0,0 +1,151 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy
import scala.collection.JavaConversions._
import scala.collection.mutable.Map
import scala.concurrent._
import akka.actor._
import akka.pattern.ask
import org.apache.log4j.{Level, Logger}
import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.{DriverState, Master}
import org.apache.spark.util.{AkkaUtils, Utils}
import akka.actor.Actor.emptyBehavior
import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
/**
* Proxy that relays messages to the driver.
*/
class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging {
var masterActor: ActorSelection = _
val timeout = AkkaUtils.askTimeout(conf)
override def preStart() = {
masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master))
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
driverArgs.cmd match {
case "launch" =>
// TODO: We could add an env variable here and intercept it in `sc.addJar` that would
// truncate filesystem paths similar to what YARN does. For now, we just require
// people call `addJar` assuming the jar is in the same directory.
val env = Map[String, String]()
System.getenv().foreach{case (k, v) => env(k) = v}
val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
driverArgs.driverOptions, env)
val driverDescription = new DriverDescription(
driverArgs.jarUrl,
driverArgs.memory,
driverArgs.cores,
driverArgs.supervise,
command)
masterActor ! RequestSubmitDriver(driverDescription)
case "kill" =>
val driverId = driverArgs.driverId
val killFuture = masterActor ! RequestKillDriver(driverId)
}
}
/* Find out driver status then exit the JVM */
def pollAndReportStatus(driverId: String) {
println(s"... waiting before polling master for driver state")
Thread.sleep(5000)
println("... polling master for driver state")
val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout)
.mapTo[DriverStatusResponse]
val statusResponse = Await.result(statusFuture, timeout)
statusResponse.found match {
case false =>
println(s"ERROR: Cluster master did not recognize $driverId")
System.exit(-1)
case true =>
println(s"State of $driverId is ${statusResponse.state.get}")
// Worker node, if present
(statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match {
case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) =>
println(s"Driver running on $hostPort ($id)")
case _ =>
}
// Exception, if present
statusResponse.exception.map { e =>
println(s"Exception from cluster was: $e")
System.exit(-1)
}
System.exit(0)
}
}
override def receive = {
case SubmitDriverResponse(success, driverId, message) =>
println(message)
if (success) pollAndReportStatus(driverId.get) else System.exit(-1)
case KillDriverResponse(driverId, success, message) =>
println(message)
if (success) pollAndReportStatus(driverId) else System.exit(-1)
case DisassociatedEvent(_, remoteAddress, _) =>
println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
System.exit(-1)
case AssociationErrorEvent(cause, _, remoteAddress, _) =>
println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
println(s"Cause was: $cause")
System.exit(-1)
}
}
/**
* Executable utility for starting and terminating drivers inside of a standalone cluster.
*/
object Client {
def main(args: Array[String]) {
val conf = new SparkConf()
val driverArgs = new ClientArguments(args)
if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
conf.set("spark.akka.logLifecycleEvents", "true")
}
conf.set("spark.akka.askTimeout", "10")
conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING"))
Logger.getRootLogger.setLevel(driverArgs.logLevel)
// TODO: See if we can initialize akka so return messages are sent back using the same TCP
// flow. Else, this (sadly) requires the DriverClient be routable from the Master.
val (actorSystem, _) = AkkaUtils.createActorSystem(
"driverClient", Utils.localHostName(), 0, false, conf)
actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf))
actorSystem.awaitTermination()
}
}

View file

@ -0,0 +1,117 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy
import java.net.URL
import scala.collection.mutable.ListBuffer
import org.apache.log4j.Level
/**
* Command-line parser for the driver client.
*/
private[spark] class ClientArguments(args: Array[String]) {
val defaultCores = 1
val defaultMemory = 512
var cmd: String = "" // 'launch' or 'kill'
var logLevel = Level.WARN
// launch parameters
var master: String = ""
var jarUrl: String = ""
var mainClass: String = ""
var supervise: Boolean = false
var memory: Int = defaultMemory
var cores: Int = defaultCores
private var _driverOptions = ListBuffer[String]()
def driverOptions = _driverOptions.toSeq
// kill parameters
var driverId: String = ""
parse(args.toList)
def parse(args: List[String]): Unit = args match {
case ("--cores" | "-c") :: value :: tail =>
cores = value.toInt
parse(tail)
case ("--memory" | "-m") :: value :: tail =>
memory = value.toInt
parse(tail)
case ("--supervise" | "-s") :: tail =>
supervise = true
parse(tail)
case ("--help" | "-h") :: tail =>
printUsageAndExit(0)
case ("--verbose" | "-v") :: tail =>
logLevel = Level.INFO
parse(tail)
case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
cmd = "launch"
try {
new URL(_jarUrl)
} catch {
case e: Exception =>
println(s"Jar url '${_jarUrl}' is not a valid URL.")
println(s"Jar must be in URL format (e.g. hdfs://XX, file://XX)")
printUsageAndExit(-1)
}
jarUrl = _jarUrl
master = _master
mainClass = _mainClass
_driverOptions ++= tail
case "kill" :: _master :: _driverId :: tail =>
cmd = "kill"
master = _master
driverId = _driverId
case _ =>
printUsageAndExit(1)
}
/**
* Print usage and exit JVM with the given exit code.
*/
def printUsageAndExit(exitCode: Int) {
// TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
// separately similar to in the YARN client.
val usage =
s"""
|Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
|Usage: DriverClient kill <active-master> <driver-id>
|
|Options:
| -c CORES, --cores CORES Number of cores to request (default: $defaultCores)
| -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory)
| -s, --supervise Whether to restart the driver on failure
| -v, --verbose Print more debugging output
""".stripMargin
System.err.println(usage)
System.exit(exitCode)
}
}

View file

@ -20,12 +20,12 @@ package org.apache.spark.deploy
import scala.collection.immutable.List
import org.apache.spark.deploy.ExecutorState.ExecutorState
import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.deploy.master.RecoveryState.MasterState
import org.apache.spark.deploy.worker.ExecutorRunner
import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
import org.apache.spark.util.Utils
private[deploy] sealed trait DeployMessage extends Serializable
/** Contains messages sent between Scheduler actor nodes. */
@ -54,7 +54,14 @@ private[deploy] object DeployMessages {
exitStatus: Option[Int])
extends DeployMessage
case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription])
case class DriverStateChanged(
driverId: String,
state: DriverState,
exception: Option[Exception])
extends DeployMessage
case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription],
driverIds: Seq[String])
case class Heartbeat(workerId: String) extends DeployMessage
@ -76,14 +83,18 @@ private[deploy] object DeployMessages {
sparkHome: String)
extends DeployMessage
// Client to Master
case class LaunchDriver(driverId: String, driverDesc: DriverDescription) extends DeployMessage
case class KillDriver(driverId: String) extends DeployMessage
// AppClient to Master
case class RegisterApplication(appDescription: ApplicationDescription)
extends DeployMessage
case class MasterChangeAcknowledged(appId: String)
// Master to Client
// Master to AppClient
case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage
@ -97,11 +108,28 @@ private[deploy] object DeployMessages {
case class ApplicationRemoved(message: String)
// Internal message in Client
// DriverClient <-> Master
case object StopClient
case class RequestSubmitDriver(driverDescription: DriverDescription) extends DeployMessage
// Master to Worker & Client
case class SubmitDriverResponse(success: Boolean, driverId: Option[String], message: String)
extends DeployMessage
case class RequestKillDriver(driverId: String) extends DeployMessage
case class KillDriverResponse(driverId: String, success: Boolean, message: String)
extends DeployMessage
case class RequestDriverStatus(driverId: String) extends DeployMessage
case class DriverStatusResponse(found: Boolean, state: Option[DriverState],
workerId: Option[String], workerHostPort: Option[String], exception: Option[Exception])
// Internal message in AppClient
case object StopAppClient
// Master to Worker & AppClient
case class MasterChanged(masterUrl: String, masterWebUiUrl: String)
@ -113,6 +141,7 @@ private[deploy] object DeployMessages {
case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
activeDrivers: Array[DriverInfo], completedDrivers: Array[DriverInfo],
status: MasterState) {
Utils.checkHost(host, "Required hostname")
@ -128,14 +157,15 @@ private[deploy] object DeployMessages {
// Worker to WorkerWebUI
case class WorkerStateResponse(host: String, port: Int, workerId: String,
executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String,
executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner],
drivers: List[DriverRunner], finishedDrivers: List[DriverRunner], masterUrl: String,
cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {
Utils.checkHost(host, "Required hostname")
assert (port > 0)
}
// Actor System to Worker
// Liveness checks in various places
case object SendHeartbeat
}

View file

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy
private[spark] class DriverDescription(
val jarUrl: String,
val mem: Int,
val cores: Int,
val supervise: Boolean,
val command: Command)
extends Serializable {
override def toString: String = s"DriverDescription (${command.mainClass})"
}

View file

@ -33,16 +33,17 @@ import org.apache.spark.deploy.master.Master
import org.apache.spark.util.AkkaUtils
/**
* The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
* and a listener for cluster events, and calls back the listener when various events occur.
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
* an app description, and a listener for cluster events, and calls back the listener when various
* events occur.
*
* @param masterUrls Each url should look like spark://host:port.
*/
private[spark] class Client(
private[spark] class AppClient(
actorSystem: ActorSystem,
masterUrls: Array[String],
appDescription: ApplicationDescription,
listener: ClientListener,
listener: AppClientListener,
conf: SparkConf)
extends Logging {
@ -155,7 +156,7 @@ private[spark] class Client(
case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) =>
logWarning(s"Could not connect to $address: $cause")
case StopClient =>
case StopAppClient =>
markDead()
sender ! true
context.stop(self)
@ -188,7 +189,7 @@ private[spark] class Client(
if (actor != null) {
try {
val timeout = AkkaUtils.askTimeout(conf)
val future = actor.ask(StopClient)(timeout)
val future = actor.ask(StopAppClient)(timeout)
Await.result(future, timeout)
} catch {
case e: TimeoutException =>

View file

@ -24,7 +24,7 @@ package org.apache.spark.deploy.client
*
* Users of this API should *not* block inside the callback methods.
*/
private[spark] trait ClientListener {
private[spark] trait AppClientListener {
def connected(appId: String): Unit
/** Disconnection may be a temporary state, as we fail over to a new Master. */

View file

@ -23,7 +23,7 @@ import org.apache.spark.deploy.{Command, ApplicationDescription}
private[spark] object TestClient {
class TestListener extends ClientListener with Logging {
class TestListener extends AppClientListener with Logging {
def connected(id: String) {
logInfo("Connected to master, got app ID " + id)
}
@ -51,7 +51,7 @@ private[spark] object TestClient {
"TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
"dummy-spark-home", "ignored")
val listener = new TestListener
val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf)
val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
client.start()
actorSystem.awaitTermination()
}

View file

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy.master
import java.util.Date
import org.apache.spark.deploy.DriverDescription
private[spark] class DriverInfo(
val startTime: Long,
val id: String,
val desc: DriverDescription,
val submitDate: Date)
extends Serializable {
@transient var state: DriverState.Value = DriverState.SUBMITTED
/* If we fail when launching the driver, the exception is stored here. */
@transient var exception: Option[Exception] = None
/* Most recent worker assigned to this driver */
@transient var worker: Option[WorkerInfo] = None
}

View file

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy.master
private[spark] object DriverState extends Enumeration {
type DriverState = Value
// SUBMITTED: Submitted but not yet scheduled on a worker
// RUNNING: Has been allocated to a worker to run
// FINISHED: Previously ran and exited cleanly
// RELAUNCHING: Exited non-zero or due to worker failure, but has not yet started running again
// UNKNOWN: The state of the driver is temporarily not known due to master failure recovery
// KILLED: A user manually killed this driver
// FAILED: The driver exited non-zero and was not supervised
// ERROR: Unable to run or restart due to an unrecoverable error (e.g. missing jar file)
val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED, ERROR = Value
}

View file

@ -19,8 +19,6 @@ package org.apache.spark.deploy.master
import java.io._
import scala.Serializable
import akka.serialization.Serialization
import org.apache.spark.Logging
@ -47,6 +45,15 @@ private[spark] class FileSystemPersistenceEngine(
new File(dir + File.separator + "app_" + app.id).delete()
}
override def addDriver(driver: DriverInfo) {
val driverFile = new File(dir + File.separator + "driver_" + driver.id)
serializeIntoFile(driverFile, driver)
}
override def removeDriver(driver: DriverInfo) {
new File(dir + File.separator + "driver_" + driver.id).delete()
}
override def addWorker(worker: WorkerInfo) {
val workerFile = new File(dir + File.separator + "worker_" + worker.id)
serializeIntoFile(workerFile, worker)
@ -56,13 +63,15 @@ private[spark] class FileSystemPersistenceEngine(
new File(dir + File.separator + "worker_" + worker.id).delete()
}
override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = {
val sortedFiles = new File(dir).listFiles().sortBy(_.getName)
val appFiles = sortedFiles.filter(_.getName.startsWith("app_"))
val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
val driverFiles = sortedFiles.filter(_.getName.startsWith("driver_"))
val drivers = driverFiles.map(deserializeFromFile[DriverInfo])
val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_"))
val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
(apps, workers)
(apps, drivers, workers)
}
private def serializeIntoFile(file: File, value: AnyRef) {

View file

@ -23,19 +23,22 @@ import java.util.Date
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Random
import akka.actor._
import akka.pattern.ask
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import akka.serialization.SerializationExtension
import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.{SparkConf, Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.MasterMessages._
import org.apache.spark.deploy.master.ui.MasterWebUI
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{AkkaUtils, Utils}
import org.apache.spark.deploy.master.DriverState.DriverState
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
import context.dispatcher // to use Akka's scheduler.schedule()
@ -43,13 +46,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
val conf = new SparkConf
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000
val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt
val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt
val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000
val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200)
val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15)
val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
var nextAppNumber = 0
val workers = new HashSet[WorkerInfo]
val idToWorker = new HashMap[String, WorkerInfo]
val actorToWorker = new HashMap[ActorRef, WorkerInfo]
@ -59,9 +61,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
val idToApp = new HashMap[String, ApplicationInfo]
val actorToApp = new HashMap[ActorRef, ApplicationInfo]
val addressToApp = new HashMap[Address, ApplicationInfo]
val waitingApps = new ArrayBuffer[ApplicationInfo]
val completedApps = new ArrayBuffer[ApplicationInfo]
var nextAppNumber = 0
val drivers = new HashSet[DriverInfo]
val completedDrivers = new ArrayBuffer[DriverInfo]
val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling
var nextDriverNumber = 0
Utils.checkHost(host, "Expected hostname")
@ -142,14 +149,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
override def receive = {
case ElectedLeader => {
val (storedApps, storedWorkers) = persistenceEngine.readPersistedData()
state = if (storedApps.isEmpty && storedWorkers.isEmpty)
val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData()
state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty)
RecoveryState.ALIVE
else
RecoveryState.RECOVERING
logInfo("I have been elected leader! New state: " + state)
if (state == RecoveryState.RECOVERING) {
beginRecovery(storedApps, storedWorkers)
beginRecovery(storedApps, storedDrivers, storedWorkers)
context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
}
}
@ -176,6 +183,69 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
}
case RequestSubmitDriver(description) => {
if (state != RecoveryState.ALIVE) {
val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
sender ! SubmitDriverResponse(false, None, msg)
} else {
logInfo("Driver submitted " + description.command.mainClass)
val driver = createDriver(description)
persistenceEngine.addDriver(driver)
waitingDrivers += driver
drivers.add(driver)
schedule()
// TODO: It might be good to instead have the submission client poll the master to determine
// the current status of the driver. For now it's simply "fire and forget".
sender ! SubmitDriverResponse(true, Some(driver.id),
s"Driver successfully submitted as ${driver.id}")
}
}
case RequestKillDriver(driverId) => {
if (state != RecoveryState.ALIVE) {
val msg = s"Can only kill drivers in ALIVE state. Current state: $state."
sender ! KillDriverResponse(driverId, success = false, msg)
} else {
logInfo("Asked to kill driver " + driverId)
val driver = drivers.find(_.id == driverId)
driver match {
case Some(d) =>
if (waitingDrivers.contains(d)) {
waitingDrivers -= d
self ! DriverStateChanged(driverId, DriverState.KILLED, None)
}
else {
// We just notify the worker to kill the driver here. The final bookkeeping occurs
// on the return path when the worker submits a state change back to the master
// to notify it that the driver was successfully killed.
d.worker.foreach { w =>
w.actor ! KillDriver(driverId)
}
}
// TODO: It would be nice for this to be a synchronous response
val msg = s"Kill request for $driverId submitted"
logInfo(msg)
sender ! KillDriverResponse(driverId, success = true, msg)
case None =>
val msg = s"Driver $driverId has already finished or does not exist"
logWarning(msg)
sender ! KillDriverResponse(driverId, success = false, msg)
}
}
}
case RequestDriverStatus(driverId) => {
(drivers ++ completedDrivers).find(_.id == driverId) match {
case Some(driver) =>
sender ! DriverStatusResponse(found = true, Some(driver.state),
driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception)
case None =>
sender ! DriverStatusResponse(found = false, None, None, None, None)
}
}
case RegisterApplication(description) => {
if (state == RecoveryState.STANDBY) {
// ignore, don't send response
@ -218,6 +288,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
}
case DriverStateChanged(driverId, state, exception) => {
state match {
case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED =>
removeDriver(driverId, state, exception)
case _ =>
throw new Exception(s"Received unexpected state update for driver $driverId: $state")
}
}
case Heartbeat(workerId) => {
idToWorker.get(workerId) match {
case Some(workerInfo) =>
@ -239,7 +318,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
if (canCompleteRecovery) { completeRecovery() }
}
case WorkerSchedulerStateResponse(workerId, executors) => {
case WorkerSchedulerStateResponse(workerId, executors, driverIds) => {
idToWorker.get(workerId) match {
case Some(worker) =>
logInfo("Worker has been re-registered: " + workerId)
@ -252,6 +331,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
worker.addExecutor(execInfo)
execInfo.copyState(exec)
}
for (driverId <- driverIds) {
drivers.find(_.id == driverId).foreach { driver =>
driver.worker = Some(worker)
driver.state = DriverState.RUNNING
worker.drivers(driverId) = driver
}
}
case None =>
logWarning("Scheduler state from unknown worker: " + workerId)
}
@ -269,7 +356,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
case RequestMasterState => {
sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
state)
drivers.toArray, completedDrivers.toArray, state)
}
case CheckForWorkerTimeOut => {
@ -285,7 +372,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
workers.count(_.state == WorkerState.UNKNOWN) == 0 &&
apps.count(_.state == ApplicationState.UNKNOWN) == 0
def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) {
def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo],
storedWorkers: Seq[WorkerInfo]) {
for (app <- storedApps) {
logInfo("Trying to recover app: " + app.id)
try {
@ -297,6 +385,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
}
for (driver <- storedDrivers) {
// Here we just read in the list of drivers. Any drivers associated with now-lost workers
// will be re-launched when we detect that the worker is missing.
drivers += driver
}
for (worker <- storedWorkers) {
logInfo("Trying to recover worker: " + worker.id)
try {
@ -320,6 +414,18 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker)
apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication)
// Reschedule drivers which were not claimed by any workers
drivers.filter(_.worker.isEmpty).foreach { d =>
logWarning(s"Driver ${d.id} was not found after master recovery")
if (d.desc.supervise) {
logWarning(s"Re-launching ${d.id}")
relaunchDriver(d)
} else {
removeDriver(d.id, DriverState.ERROR, None)
logWarning(s"Did not re-launch ${d.id} because it was not supervised")
}
}
state = RecoveryState.ALIVE
schedule()
logInfo("Recovery complete - resuming operations!")
@ -340,6 +446,18 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
*/
def schedule() {
if (state != RecoveryState.ALIVE) { return }
// First schedule drivers, they take strict precedence over applications
val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers
for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) {
for (driver <- waitingDrivers) {
if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) {
launchDriver(worker, driver)
waitingDrivers -= driver
}
}
}
// Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
// in the queue, then the second app, etc.
if (spreadOutApps) {
@ -426,9 +544,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
exec.id, ExecutorState.LOST, Some("worker lost"), None)
exec.application.removeExecutor(exec)
}
for (driver <- worker.drivers.values) {
if (driver.desc.supervise) {
logInfo(s"Re-launching ${driver.id}")
relaunchDriver(driver)
} else {
logInfo(s"Not re-launching ${driver.id} because it was not supervised")
removeDriver(driver.id, DriverState.ERROR, None)
}
}
persistenceEngine.removeWorker(worker)
}
def relaunchDriver(driver: DriverInfo) {
driver.worker = None
driver.state = DriverState.RELAUNCHING
waitingDrivers += driver
schedule()
}
def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
val now = System.currentTimeMillis()
val date = new Date(now)
@ -508,6 +642,41 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
}
}
def newDriverId(submitDate: Date): String = {
val appId = "driver-%s-%04d".format(DATE_FORMAT.format(submitDate), nextDriverNumber)
nextDriverNumber += 1
appId
}
def createDriver(desc: DriverDescription): DriverInfo = {
val now = System.currentTimeMillis()
val date = new Date(now)
new DriverInfo(now, newDriverId(date), desc, date)
}
def launchDriver(worker: WorkerInfo, driver: DriverInfo) {
logInfo("Launching driver " + driver.id + " on worker " + worker.id)
worker.addDriver(driver)
driver.worker = Some(worker)
worker.actor ! LaunchDriver(driver.id, driver.desc)
driver.state = DriverState.RUNNING
}
def removeDriver(driverId: String, finalState: DriverState, exception: Option[Exception]) {
drivers.find(d => d.id == driverId) match {
case Some(driver) =>
logInfo(s"Removing driver: $driverId")
drivers -= driver
completedDrivers += driver
persistenceEngine.removeDriver(driver)
driver.state = finalState
driver.exception = exception
driver.worker.foreach(w => w.removeDriver(driver))
case None =>
logWarning(s"Asked to remove unknown driver: $driverId")
}
}
}
private[spark] object Master {

View file

@ -35,11 +35,15 @@ private[spark] trait PersistenceEngine {
def removeWorker(worker: WorkerInfo)
def addDriver(driver: DriverInfo)
def removeDriver(driver: DriverInfo)
/**
* Returns the persisted data sorted by their respective ids (which implies that they're
* sorted by time of creation).
*/
def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo])
def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo])
def close() {}
}
@ -49,5 +53,8 @@ private[spark] class BlackHolePersistenceEngine extends PersistenceEngine {
override def removeApplication(app: ApplicationInfo) {}
override def addWorker(worker: WorkerInfo) {}
override def removeWorker(worker: WorkerInfo) {}
override def readPersistedData() = (Nil, Nil)
override def addDriver(driver: DriverInfo) {}
override def removeDriver(driver: DriverInfo) {}
override def readPersistedData() = (Nil, Nil, Nil)
}

View file

@ -17,8 +17,10 @@
package org.apache.spark.deploy.master
import akka.actor.ActorRef
import scala.collection.mutable
import akka.actor.ActorRef
import org.apache.spark.util.Utils
private[spark] class WorkerInfo(
@ -35,7 +37,8 @@ private[spark] class WorkerInfo(
Utils.checkHost(host, "Expected hostname")
assert (port > 0)
@transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
@transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // executorId => info
@transient var drivers: mutable.HashMap[String, DriverInfo] = _ // driverId => info
@transient var state: WorkerState.Value = _
@transient var coresUsed: Int = _
@transient var memoryUsed: Int = _
@ -54,6 +57,7 @@ private[spark] class WorkerInfo(
private def init() {
executors = new mutable.HashMap
drivers = new mutable.HashMap
state = WorkerState.ALIVE
coresUsed = 0
memoryUsed = 0
@ -83,6 +87,18 @@ private[spark] class WorkerInfo(
executors.values.exists(_.application == app)
}
def addDriver(driver: DriverInfo) {
drivers(driver.id) = driver
memoryUsed += driver.desc.mem
coresUsed += driver.desc.cores
}
def removeDriver(driver: DriverInfo) {
drivers -= driver.id
memoryUsed -= driver.desc.mem
coresUsed -= driver.desc.cores
}
def webUiAddress : String = {
"http://" + this.publicAddress + ":" + this.webUiPort
}

View file

@ -49,6 +49,14 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
zk.delete(WORKING_DIR + "/app_" + app.id)
}
override def addDriver(driver: DriverInfo) {
serializeIntoFile(WORKING_DIR + "/driver_" + driver.id, driver)
}
override def removeDriver(driver: DriverInfo) {
zk.delete(WORKING_DIR + "/driver_" + driver.id)
}
override def addWorker(worker: WorkerInfo) {
serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker)
}
@ -61,13 +69,15 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
zk.close()
}
override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = {
val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted
val appFiles = sortedFiles.filter(_.startsWith("app_"))
val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
val driverFiles = sortedFiles.filter(_.startsWith("driver_"))
val drivers = driverFiles.map(deserializeFromFile[DriverInfo])
val workerFiles = sortedFiles.filter(_.startsWith("worker_"))
val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
(apps, workers)
(apps, drivers, workers)
}
private def serializeIntoFile(path: String, value: AnyRef) {

View file

@ -67,11 +67,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
<li><strong>User:</strong> {app.desc.user}</li>
<li><strong>Cores:</strong>
{
if (app.desc.maxCores == Integer.MAX_VALUE) {
if (app.desc.maxCores == None) {
"Unlimited (%s granted)".format(app.coresGranted)
} else {
"%s (%s granted, %s left)".format(
app.desc.maxCores, app.coresGranted, app.coresLeft)
app.desc.maxCores.get, app.coresGranted, app.coresLeft)
}
}
</li>

View file

@ -18,6 +18,7 @@
package org.apache.spark.deploy.master.ui
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.xml.Node
import akka.pattern.ask
@ -26,7 +27,7 @@ import net.liftweb.json.JsonAST.JValue
import org.apache.spark.deploy.{DeployWebUI, JsonProtocol}
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
@ -56,6 +57,16 @@ private[spark] class IndexPage(parent: MasterWebUI) {
val completedApps = state.completedApps.sortBy(_.endTime).reverse
val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class")
val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
// For now we only show driver information if the user has submitted drivers to the cluster.
// This is until we integrate the notion of drivers and applications in the UI.
def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
val content =
<div class="row-fluid">
<div class="span12">
@ -70,6 +81,9 @@ private[spark] class IndexPage(parent: MasterWebUI) {
<li><strong>Applications:</strong>
{state.activeApps.size} Running,
{state.completedApps.size} Completed </li>
<li><strong>Drivers:</strong>
{state.activeDrivers.size} Running,
{state.completedDrivers.size} Completed </li>
</ul>
</div>
</div>
@ -84,17 +98,39 @@ private[spark] class IndexPage(parent: MasterWebUI) {
<div class="row-fluid">
<div class="span12">
<h4> Running Applications </h4>
{activeAppsTable}
</div>
</div>
<div>
{if (hasDrivers)
<div class="row-fluid">
<div class="span12">
<h4> Running Drivers </h4>
{activeDriversTable}
</div>
</div>
}
</div>
<div class="row-fluid">
<div class="span12">
<h4> Completed Applications </h4>
{completedAppsTable}
</div>
</div>
<div>
{if (hasDrivers)
<div class="row-fluid">
<div class="span12">
<h4> Completed Drivers </h4>
{completedDriversTable}
</div>
</div>
}
</div>;
UIUtils.basicSparkPage(content, "Spark Master at " + state.uri)
}
@ -134,4 +170,20 @@ private[spark] class IndexPage(parent: MasterWebUI) {
<td>{DeployWebUI.formatDuration(app.duration)}</td>
</tr>
}
def driverRow(driver: DriverInfo): Seq[Node] = {
<tr>
<td>{driver.id} </td>
<td>{driver.submitDate}</td>
<td>{driver.worker.map(w => <a href={w.webUiAddress}>{w.id.toString}</a>).getOrElse("None")}</td>
<td>{driver.state}</td>
<td sorttable_customkey={driver.desc.cores.toString}>
{driver.desc.cores}
</td>
<td sorttable_customkey={driver.desc.mem.toString}>
{Utils.megabytesToString(driver.desc.mem.toLong)}
</td>
<td>{driver.desc.command.arguments(1)}</td>
</tr>
}
}

View file

@ -0,0 +1,63 @@
package org.apache.spark.deploy.worker
import java.io.{File, FileOutputStream, IOException, InputStream}
import java.lang.System._
import org.apache.spark.Logging
import org.apache.spark.deploy.Command
import org.apache.spark.util.Utils
/**
** Utilities for running commands with the spark classpath.
*/
object CommandUtils extends Logging {
private[spark] def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = {
val runner = getEnv("JAVA_HOME", command).map(_ + "/bin/java").getOrElse("java")
// SPARK-698: do not call the run.cmd script, as process.destroy()
// fails to kill a process tree on Windows
Seq(runner) ++ buildJavaOpts(command, memory, sparkHome) ++ Seq(command.mainClass) ++
command.arguments
}
private def getEnv(key: String, command: Command): Option[String] =
command.environment.get(key).orElse(Option(System.getenv(key)))
/**
* Attention: this must always be aligned with the environment variables in the run scripts and
* the way the JAVA_OPTS are assembled there.
*/
def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = {
val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command)
.map(p => List("-Djava.library.path=" + p))
.getOrElse(Nil)
val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil)
val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil)
val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M")
// Figure out our classpath with the external compute-classpath script
val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
val classPath = Utils.executeAndGetOutput(
Seq(sparkHome + "/bin/compute-classpath" + ext),
extraEnvironment=command.environment)
Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts
}
/** Spawn a thread that will redirect a given stream to a file */
def redirectStream(in: InputStream, file: File) {
val out = new FileOutputStream(file, true)
// TODO: It would be nice to add a shutdown hook here that explains why the output is
// terminating. Otherwise if the worker dies the executor logs will silently stop.
new Thread("redirect output to " + file) {
override def run() {
try {
Utils.copyStream(in, out, true)
} catch {
case e: IOException =>
logInfo("Redirection to " + file + " closed: " + e.getMessage)
}
}
}.start()
}
}

View file

@ -0,0 +1,234 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy.worker
import java.io._
import scala.collection.JavaConversions._
import scala.collection.mutable.Map
import akka.actor.ActorRef
import com.google.common.base.Charsets
import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileUtil, Path}
import org.apache.spark.Logging
import org.apache.spark.deploy.{Command, DriverDescription}
import org.apache.spark.deploy.DeployMessages.DriverStateChanged
import org.apache.spark.deploy.master.DriverState
import org.apache.spark.deploy.master.DriverState.DriverState
/**
* Manages the execution of one driver, including automatically restarting the driver on failure.
*/
private[spark] class DriverRunner(
val driverId: String,
val workDir: File,
val sparkHome: File,
val driverDesc: DriverDescription,
val worker: ActorRef,
val workerUrl: String)
extends Logging {
@volatile var process: Option[Process] = None
@volatile var killed = false
// Populated once finished
var finalState: Option[DriverState] = None
var finalException: Option[Exception] = None
var finalExitCode: Option[Int] = None
// Decoupled for testing
private[deploy] def setClock(_clock: Clock) = clock = _clock
private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
private var clock = new Clock {
def currentTimeMillis(): Long = System.currentTimeMillis()
}
private var sleeper = new Sleeper {
def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed})
}
/** Starts a thread to run and manage the driver. */
def start() = {
new Thread("DriverRunner for " + driverId) {
override def run() {
try {
val driverDir = createWorkingDirectory()
val localJarFilename = downloadUserJar(driverDir)
// Make sure user application jar is on the classpath
// TODO: If we add ability to submit multiple jars they should also be added here
val env = Map(driverDesc.command.environment.toSeq: _*)
env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename"
val newCommand = Command(driverDesc.command.mainClass,
driverDesc.command.arguments.map(substituteVariables), env)
val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
sparkHome.getAbsolutePath)
launchDriver(command, env, driverDir, driverDesc.supervise)
}
catch {
case e: Exception => finalException = Some(e)
}
val state =
if (killed) { DriverState.KILLED }
else if (finalException.isDefined) { DriverState.ERROR }
else {
finalExitCode match {
case Some(0) => DriverState.FINISHED
case _ => DriverState.FAILED
}
}
finalState = Some(state)
worker ! DriverStateChanged(driverId, state, finalException)
}
}.start()
}
/** Terminate this driver (or prevent it from ever starting if not yet started) */
def kill() {
synchronized {
process.foreach(p => p.destroy())
killed = true
}
}
/** Replace variables in a command argument passed to us */
private def substituteVariables(argument: String): String = argument match {
case "{{WORKER_URL}}" => workerUrl
case other => other
}
/**
* Creates the working directory for this driver.
* Will throw an exception if there are errors preparing the directory.
*/
private def createWorkingDirectory(): File = {
val driverDir = new File(workDir, driverId)
if (!driverDir.exists() && !driverDir.mkdirs()) {
throw new IOException("Failed to create directory " + driverDir)
}
driverDir
}
/**
* Download the user jar into the supplied directory and return its local path.
* Will throw an exception if there are errors downloading the jar.
*/
private def downloadUserJar(driverDir: File): String = {
val jarPath = new Path(driverDesc.jarUrl)
val emptyConf = new Configuration()
val jarFileSystem = jarPath.getFileSystem(emptyConf)
val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
val jarFileName = jarPath.getName
val localJarFile = new File(driverDir, jarFileName)
val localJarFilename = localJarFile.getAbsolutePath
if (!localJarFile.exists()) { // May already exist if running multiple workers on one node
logInfo(s"Copying user jar $jarPath to $destPath")
FileUtil.copy(jarFileSystem, jarPath, destPath, false, emptyConf)
}
if (!localJarFile.exists()) { // Verify copy succeeded
throw new Exception(s"Did not see expected jar $jarFileName in $driverDir")
}
localJarFilename
}
private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File,
supervise: Boolean) {
val builder = new ProcessBuilder(command: _*).directory(baseDir)
envVars.map{ case(k,v) => builder.environment().put(k, v) }
def initialize(process: Process) = {
// Redirect stdout and stderr to files
val stdout = new File(baseDir, "stdout")
CommandUtils.redirectStream(process.getInputStream, stdout)
val stderr = new File(baseDir, "stderr")
val header = "Launch Command: %s\n%s\n\n".format(
command.mkString("\"", "\" \"", "\""), "=" * 40)
Files.append(header, stderr, Charsets.UTF_8)
CommandUtils.redirectStream(process.getErrorStream, stderr)
}
runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise)
}
private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit,
supervise: Boolean) {
// Time to wait between submission retries.
var waitSeconds = 1
// A run of this many seconds resets the exponential back-off.
val successfulRunDuration = 5
var keepTrying = !killed
while (keepTrying) {
logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\""))
synchronized {
if (killed) { return }
process = Some(command.start())
initialize(process.get)
}
val processStart = clock.currentTimeMillis()
val exitCode = process.get.waitFor()
if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
waitSeconds = 1
}
if (supervise && exitCode != 0 && !killed) {
logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
sleeper.sleep(waitSeconds)
waitSeconds = waitSeconds * 2 // exponential back-off
}
keepTrying = supervise && exitCode != 0 && !killed
finalExitCode = Some(exitCode)
}
}
}
private[deploy] trait Clock {
def currentTimeMillis(): Long
}
private[deploy] trait Sleeper {
def sleep(seconds: Int)
}
// Needed because ProcessBuilder is a final class and cannot be mocked
private[deploy] trait ProcessBuilderLike {
def start(): Process
def command: Seq[String]
}
private[deploy] object ProcessBuilderLike {
def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike {
def start() = processBuilder.start()
def command = processBuilder.command()
}
}

View file

@ -0,0 +1,31 @@
package org.apache.spark.deploy.worker
import akka.actor._
import org.apache.spark.SparkConf
import org.apache.spark.util.{AkkaUtils, Utils}
/**
* Utility object for launching driver programs such that they share fate with the Worker process.
*/
object DriverWrapper {
def main(args: Array[String]) {
args.toList match {
case workerUrl :: mainClass :: extraArgs =>
val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
Utils.localHostName(), 0, false, new SparkConf())
actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
// Delegate to supplied main class
val clazz = Class.forName(args(1))
val mainMethod = clazz.getMethod("main", classOf[Array[String]])
mainMethod.invoke(null, extraArgs.toArray[String])
actorSystem.shutdown()
case _ =>
System.err.println("Usage: DriverWrapper <workerUrl> <driverMainClass> [options]")
System.exit(-1)
}
}
}

View file

@ -18,17 +18,15 @@
package org.apache.spark.deploy.worker
import java.io._
import java.lang.System.getenv
import akka.actor.ActorRef
import com.google.common.base.Charsets
import com.google.common.io.Files
import org.apache.spark.{Logging}
import org.apache.spark.deploy.{ExecutorState, ApplicationDescription}
import org.apache.spark.Logging
import org.apache.spark.deploy.{ExecutorState, ApplicationDescription, Command}
import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
import org.apache.spark.util.Utils
/**
* Manages the execution of one executor process.
@ -44,16 +42,17 @@ private[spark] class ExecutorRunner(
val host: String,
val sparkHome: File,
val workDir: File,
val workerUrl: String,
var state: ExecutorState.Value)
extends Logging {
val fullId = appId + "/" + execId
var workerThread: Thread = null
var process: Process = null
var shutdownHook: Thread = null
private def getAppEnv(key: String): Option[String] =
appDesc.command.environment.get(key).orElse(Option(getenv(key)))
// NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might
// make sense to remove this in the future.
var shutdownHook: Thread = null
def start() {
workerThread = new Thread("ExecutorRunner for " + fullId) {
@ -92,55 +91,17 @@ private[spark] class ExecutorRunner(
/** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */
def substituteVariables(argument: String): String = argument match {
case "{{WORKER_URL}}" => workerUrl
case "{{EXECUTOR_ID}}" => execId.toString
case "{{HOSTNAME}}" => host
case "{{CORES}}" => cores.toString
case other => other
}
def buildCommandSeq(): Seq[String] = {
val command = appDesc.command
val runner = getAppEnv("JAVA_HOME").map(_ + "/bin/java").getOrElse("java")
// SPARK-698: do not call the run.cmd script, as process.destroy()
// fails to kill a process tree on Windows
Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++
(command.arguments ++ Seq(appId)).map(substituteVariables)
}
/**
* Attention: this must always be aligned with the environment variables in the run scripts and
* the way the JAVA_OPTS are assembled there.
*/
def buildJavaOpts(): Seq[String] = {
val libraryOpts = getAppEnv("SPARK_LIBRARY_PATH")
.map(p => List("-Djava.library.path=" + p))
.getOrElse(Nil)
val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil)
val userOpts = getAppEnv("SPARK_JAVA_OPTS").map(Utils.splitCommandString).getOrElse(Nil)
val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M")
// Figure out our classpath with the external compute-classpath script
val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
val classPath = Utils.executeAndGetOutput(
Seq(sparkHome + "/bin/compute-classpath" + ext),
extraEnvironment=appDesc.command.environment)
Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts
}
/** Spawn a thread that will redirect a given stream to a file */
def redirectStream(in: InputStream, file: File) {
val out = new FileOutputStream(file, true)
new Thread("redirect output to " + file) {
override def run() {
try {
Utils.copyStream(in, out, true)
} catch {
case e: IOException =>
logInfo("Redirection to " + file + " closed: " + e.getMessage)
}
}
}.start()
def getCommandSeq = {
val command = Command(appDesc.command.mainClass,
appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment)
CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
}
/**
@ -155,7 +116,7 @@ private[spark] class ExecutorRunner(
}
// Launch the process
val command = buildCommandSeq()
val command = getCommandSeq
logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
val builder = new ProcessBuilder(command: _*).directory(executorDir)
val env = builder.environment()
@ -172,11 +133,11 @@ private[spark] class ExecutorRunner(
// Redirect its stdout and stderr to files
val stdout = new File(executorDir, "stdout")
redirectStream(process.getInputStream, stdout)
CommandUtils.redirectStream(process.getInputStream, stdout)
val stderr = new File(executorDir, "stderr")
Files.write(header, stderr, Charsets.UTF_8)
redirectStream(process.getErrorStream, stderr)
CommandUtils.redirectStream(process.getErrorStream, stderr)
// Wait for it to exit; this is actually a bad thing if it happens, because we expect to run
// long-lived processes only. However, in the future, we might restart the executor a few

View file

@ -26,10 +26,12 @@ import scala.concurrent.duration._
import akka.actor._
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.spark.{Logging, SparkConf, SparkException}
import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
import org.apache.spark.deploy.master.{DriverState, Master}
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.deploy.worker.ui.WorkerWebUI
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{AkkaUtils, Utils}
@ -44,6 +46,8 @@ private[spark] class Worker(
cores: Int,
memory: Int,
masterUrls: Array[String],
actorSystemName: String,
actorName: String,
workDirPath: String = null,
val conf: SparkConf)
extends Actor with Logging {
@ -55,7 +59,7 @@ private[spark] class Worker(
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs
// Send a heartbeat every (heartbeat timeout) / 4 milliseconds
val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4
val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3
@ -68,6 +72,7 @@ private[spark] class Worker(
var masterAddress: Address = null
var activeMasterUrl: String = ""
var activeMasterWebUiUrl : String = ""
val akkaUrl = "akka.tcp://%s@%s:%s/user/%s".format(actorSystemName, host, port, actorName)
@volatile var registered = false
@volatile var connected = false
val workerId = generateWorkerId()
@ -75,6 +80,9 @@ private[spark] class Worker(
var workDir: File = null
val executors = new HashMap[String, ExecutorRunner]
val finishedExecutors = new HashMap[String, ExecutorRunner]
val drivers = new HashMap[String, DriverRunner]
val finishedDrivers = new HashMap[String, DriverRunner]
val publicAddress = {
val envVar = System.getenv("SPARK_PUBLIC_DNS")
if (envVar != null) envVar else host
@ -185,7 +193,10 @@ private[spark] class Worker(
val execs = executors.values.
map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
sender ! WorkerSchedulerStateResponse(workerId, execs.toList)
sender ! WorkerSchedulerStateResponse(workerId, execs.toList, drivers.keys.toSeq)
case Heartbeat =>
logInfo(s"Received heartbeat from driver ${sender.path}")
case RegisterWorkerFailed(message) =>
if (!registered) {
@ -199,7 +210,7 @@ private[spark] class Worker(
} else {
logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
self, workerId, host, new File(execSparkHome_), workDir, ExecutorState.RUNNING)
self, workerId, host, new File(execSparkHome_), workDir, akkaUrl, ExecutorState.RUNNING)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
@ -219,8 +230,8 @@ private[spark] class Worker(
logInfo("Executor " + fullId + " finished with state " + state +
message.map(" message " + _).getOrElse("") +
exitStatus.map(" exitStatus " + _).getOrElse(""))
finishedExecutors(fullId) = executor
executors -= fullId
finishedExecutors(fullId) = executor
coresUsed -= executor.cores
memoryUsed -= executor.memory
}
@ -239,13 +250,52 @@ private[spark] class Worker(
}
}
case LaunchDriver(driverId, driverDesc) => {
logInfo(s"Asked to launch driver $driverId")
val driver = new DriverRunner(driverId, workDir, sparkHome, driverDesc, self, akkaUrl)
drivers(driverId) = driver
driver.start()
coresUsed += driverDesc.cores
memoryUsed += driverDesc.mem
}
case KillDriver(driverId) => {
logInfo(s"Asked to kill driver $driverId")
drivers.get(driverId) match {
case Some(runner) =>
runner.kill()
case None =>
logError(s"Asked to kill unknown driver $driverId")
}
}
case DriverStateChanged(driverId, state, exception) => {
state match {
case DriverState.ERROR =>
logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
case DriverState.FINISHED =>
logInfo(s"Driver $driverId exited successfully")
case DriverState.KILLED =>
logInfo(s"Driver $driverId was killed by user")
}
masterLock.synchronized {
master ! DriverStateChanged(driverId, state, exception)
}
val driver = drivers.remove(driverId).get
finishedDrivers(driverId) = driver
memoryUsed -= driver.driverDesc.mem
coresUsed -= driver.driverDesc.cores
}
case x: DisassociatedEvent if x.remoteAddress == masterAddress =>
logInfo(s"$x Disassociated !")
masterDisconnected()
case RequestWorkerState => {
sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
finishedExecutors.values.toList, activeMasterUrl, cores, memory,
finishedExecutors.values.toList, drivers.values.toList,
finishedDrivers.values.toList, activeMasterUrl, cores, memory,
coresUsed, memoryUsed, activeMasterWebUiUrl)
}
}
@ -282,10 +332,11 @@ private[spark] object Worker {
// The LocalSparkCluster runs multiple local sparkWorkerX actor systems
val conf = new SparkConf
val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
val actorName = "Worker"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
conf = conf)
actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
masterUrls, workDir, conf), name = "Worker")
masterUrls, systemName, actorName, workDir, conf), name = actorName)
(actorSystem, boundPort)
}

View file

@ -0,0 +1,55 @@
package org.apache.spark.deploy.worker
import akka.actor.{Actor, Address, AddressFromURIString}
import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.spark.Logging
import org.apache.spark.deploy.DeployMessages.SendHeartbeat
/**
* Actor which connects to a worker process and terminates the JVM if the connection is severed.
* Provides fate sharing between a worker and its associated child processes.
*/
private[spark] class WorkerWatcher(workerUrl: String) extends Actor
with Logging {
override def preStart() {
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
logInfo(s"Connecting to worker $workerUrl")
val worker = context.actorSelection(workerUrl)
worker ! SendHeartbeat // need to send a message here to initiate connection
}
// Used to avoid shutting down JVM during tests
private[deploy] var isShutDown = false
private[deploy] def setTesting(testing: Boolean) = isTesting = testing
private var isTesting = false
// Lets us filter events only from the worker's actor system
private val expectedHostPort = AddressFromURIString(workerUrl).hostPort
private def isWorker(address: Address) = address.hostPort == expectedHostPort
def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1)
override def receive = {
case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
logInfo(s"Successfully connected to $workerUrl")
case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound)
if isWorker(remoteAddress) =>
// These logs may not be seen if the worker (and associated pipe) has died
logError(s"Could not initialize connection to worker $workerUrl. Exiting.")
logError(s"Error was: $cause")
exitNonZero()
case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
// This log message will never be seen
logError(s"Lost connection to worker actor $workerUrl. Exiting.")
exitNonZero()
case e: AssociationEvent =>
// pass through association events relating to other remote actor systems
case e => logWarning(s"Received unexpected actor system event: $e")
}
}

View file

@ -17,24 +17,20 @@
package org.apache.spark.deploy.worker.ui
import javax.servlet.http.HttpServletRequest
import scala.concurrent.Await
import scala.xml.Node
import scala.concurrent.duration._
import scala.concurrent.Await
import akka.pattern.ask
import javax.servlet.http.HttpServletRequest
import net.liftweb.json.JsonAST.JValue
import org.apache.spark.deploy.JsonProtocol
import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
import org.apache.spark.deploy.worker.ExecutorRunner
import org.apache.spark.deploy.master.DriverState
import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
private[spark] class IndexPage(parent: WorkerWebUI) {
val workerActor = parent.worker.self
val worker = parent.worker
@ -56,6 +52,16 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
val finishedExecutorTable =
UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes")
val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers)
val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers)
// For now we only show driver information if the user has submitted drivers to the cluster.
// This is until we integrate the notion of drivers and applications in the UI.
def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0
val content =
<div class="row-fluid"> <!-- Worker Details -->
<div class="span12">
@ -79,11 +85,33 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
</div>
</div>
<div>
{if (hasDrivers)
<div class="row-fluid"> <!-- Running Drivers -->
<div class="span12">
<h4> Running Drivers {workerState.drivers.size} </h4>
{runningDriverTable}
</div>
</div>
}
</div>
<div class="row-fluid"> <!-- Finished Executors -->
<div class="span12">
<h4> Finished Executors </h4>
{finishedExecutorTable}
</div>
</div>
<div>
{if (hasDrivers)
<div class="row-fluid"> <!-- Finished Drivers -->
<div class="span12">
<h4> Finished Drivers </h4>
{finishedDriverTable}
</div>
</div>
}
</div>;
UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format(
@ -111,6 +139,27 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
.format(executor.appId, executor.execId)}>stderr</a>
</td>
</tr>
}
def driverRow(driver: DriverRunner): Seq[Node] = {
<tr>
<td>{driver.driverId}</td>
<td>{driver.driverDesc.command.arguments(1)}</td>
<td>{driver.finalState.getOrElse(DriverState.RUNNING)}</td>
<td sorttable_customkey={driver.driverDesc.cores.toString}>
{driver.driverDesc.cores.toString}
</td>
<td sorttable_customkey={driver.driverDesc.mem.toString}>
{Utils.megabytesToString(driver.driverDesc.mem)}
</td>
<td>
<a href={s"logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>
<a href={s"logPage?driverId=${driver.driverId}&logType=stderr"}>stderr</a>
</td>
<td>
{driver.finalException.getOrElse("")}
</td>
</tr>
}
}

View file

@ -69,30 +69,48 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
def log(request: HttpServletRequest): String = {
val defaultBytes = 100 * 1024
val appId = request.getParameter("appId")
val executorId = request.getParameter("executorId")
val appId = Option(request.getParameter("appId"))
val executorId = Option(request.getParameter("executorId"))
val driverId = Option(request.getParameter("driverId"))
val logType = request.getParameter("logType")
val offset = Option(request.getParameter("offset")).map(_.toLong)
val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes)
val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType)
val path = (appId, executorId, driverId) match {
case (Some(a), Some(e), None) =>
s"${workDir.getPath}/$appId/$executorId/$logType"
case (None, None, Some(d)) =>
s"${workDir.getPath}/$driverId/$logType"
case _ =>
throw new Exception("Request must specify either application or driver identifiers")
}
val (startByte, endByte) = getByteRange(path, offset, byteLength)
val file = new File(path)
val logLength = file.length
val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n"
.format(startByte, endByte, logLength, appId, executorId, logType)
val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n"
pre + Utils.offsetBytes(path, startByte, endByte)
}
def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = {
val defaultBytes = 100 * 1024
val appId = request.getParameter("appId")
val executorId = request.getParameter("executorId")
val appId = Option(request.getParameter("appId"))
val executorId = Option(request.getParameter("executorId"))
val driverId = Option(request.getParameter("driverId"))
val logType = request.getParameter("logType")
val offset = Option(request.getParameter("offset")).map(_.toLong)
val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes)
val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType)
val (path, params) = (appId, executorId, driverId) match {
case (Some(a), Some(e), None) =>
(s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e")
case (None, None, Some(d)) =>
(s"${workDir.getPath}/$d/$logType", s"driverId=$d")
case _ =>
throw new Exception("Request must specify either application or driver identifiers")
}
val (startByte, endByte) = getByteRange(path, offset, byteLength)
val file = new File(path)
@ -106,9 +124,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
val backButton =
if (startByte > 0) {
<a href={"?appId=%s&executorId=%s&logType=%s&offset=%s&byteLength=%s"
.format(appId, executorId, logType, math.max(startByte-byteLength, 0),
byteLength)}>
<a href={"?%s&logType=%s&offset=%s&byteLength=%s"
.format(params, logType, math.max(startByte-byteLength, 0), byteLength)}>
<button type="button" class="btn btn-default">
Previous {Utils.bytesToString(math.min(byteLength, startByte))}
</button>
@ -122,8 +139,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
val nextButton =
if (endByte < logLength) {
<a href={"?appId=%s&executorId=%s&logType=%s&offset=%s&byteLength=%s".
format(appId, executorId, logType, endByte, byteLength)}>
<a href={"?%s&logType=%s&offset=%s&byteLength=%s".
format(params, logType, endByte, byteLength)}>
<button type="button" class="btn btn-default">
Next {Utils.bytesToString(math.min(byteLength, logLength-endByte))}
</button>

View file

@ -24,8 +24,9 @@ import akka.remote._
import org.apache.spark.{SparkConf, SparkContext, Logging}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.deploy.worker.WorkerWatcher
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
import org.apache.spark.util.{Utils, AkkaUtils}
import org.apache.spark.util.{AkkaUtils, Utils}
private[spark] class CoarseGrainedExecutorBackend(
driverUrl: String,
@ -91,7 +92,8 @@ private[spark] class CoarseGrainedExecutorBackend(
}
private[spark] object CoarseGrainedExecutorBackend {
def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
def run(driverUrl: String, executorId: String, hostname: String, cores: Int,
workerUrl: Option[String]) {
// Debug code
Utils.checkHost(hostname)
@ -101,21 +103,27 @@ private[spark] object CoarseGrainedExecutorBackend {
indestructible = true, conf = new SparkConf)
// set it
val sparkHostPort = hostname + ":" + boundPort
// conf.set("spark.hostPort", sparkHostPort)
actorSystem.actorOf(
Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
name = "Executor")
workerUrl.foreach{ url =>
actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher")
}
actorSystem.awaitTermination()
}
def main(args: Array[String]) {
if (args.length < 4) {
//the reason we allow the last appid argument is to make it easy to kill rogue executors
System.err.println(
"Usage: CoarseGrainedExecutorBackend <driverUrl> <executorId> <hostname> <cores> " +
"[<appid>]")
System.exit(1)
args.length match {
case x if x < 4 =>
System.err.println(
// Worker url is used in spark standalone mode to enforce fate-sharing with worker
"Usage: CoarseGrainedExecutorBackend <driverUrl> <executorId> <hostname> " +
"<cores> [<workerUrl>]")
System.exit(1)
case 4 =>
run(args(0), args(1), args(2), args(3).toInt, None)
case x if x > 4 =>
run(args(0), args(1), args(2), args(3).toInt, Some(args(4)))
}
run(args(0), args(1), args(2), args(3).toInt)
}
}

View file

@ -57,7 +57,7 @@ private[spark] class Executor(
Utils.setCustomHostname(slaveHostname)
// Set spark.* properties from executor arg
val conf = new SparkConf(false)
val conf = new SparkConf(true)
conf.setAll(properties)
// If we are in yarn mode, systems can have different disk layouts so we must set it
@ -279,6 +279,11 @@ private[spark] class Executor(
//System.exit(1)
}
} finally {
// TODO: Unregister shuffle memory only for ShuffleMapTask
val shuffleMemoryMap = env.shuffleMemoryMap
shuffleMemoryMap.synchronized {
shuffleMemoryMap.remove(Thread.currentThread().getId)
}
runningTasks.remove(taskId)
}
}

View file

@ -71,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
override def compressedOutputStream(s: OutputStream): OutputStream = {
val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt
val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768)
new SnappyOutputStream(s, blockSize)
}

View file

@ -46,7 +46,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId:
throw new Exception("Max chunk size is " + maxChunkSize)
}
if (size == 0 && gotChunkForSendingOnce == false) {
if (size == 0 && !gotChunkForSendingOnce) {
val newChunk = new MessageChunk(
new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null)
gotChunkForSendingOnce = true

View file

@ -330,7 +330,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector,
// Is highly unlikely unless there was an unclean close of socket, etc
registerInterest()
logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending")
return true
true
} catch {
case e: Exception => {
logWarning("Error finishing connection to " + address, e)
@ -385,7 +385,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector,
}
}
// should not happen - to keep scala compiler happy
return true
true
}
// This is a hack to determine if remote socket was closed or not.
@ -559,7 +559,7 @@ private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : S
}
}
// should not happen - to keep scala compiler happy
return true
true
}
def onReceive(callback: (Connection, Message) => Unit) {onReceiveCallback = callback}

View file

@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi
private val selector = SelectorProvider.provider.openSelector()
private val handleMessageExecutor = new ThreadPoolExecutor(
conf.get("spark.core.connection.handler.threads.min", "20").toInt,
conf.get("spark.core.connection.handler.threads.max", "60").toInt,
conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
conf.getInt("spark.core.connection.handler.threads.min", 20),
conf.getInt("spark.core.connection.handler.threads.max", 60),
conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
private val handleReadWriteExecutor = new ThreadPoolExecutor(
conf.get("spark.core.connection.io.threads.min", "4").toInt,
conf.get("spark.core.connection.io.threads.max", "32").toInt,
conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
conf.getInt("spark.core.connection.io.threads.min", 4),
conf.getInt("spark.core.connection.io.threads.max", 32),
conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
// Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap
private val handleConnectExecutor = new ThreadPoolExecutor(
conf.get("spark.core.connection.connect.threads.min", "1").toInt,
conf.get("spark.core.connection.connect.threads.max", "8").toInt,
conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
conf.getInt("spark.core.connection.connect.threads.min", 1),
conf.getInt("spark.core.connection.connect.threads.max", 8),
conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
private val serverChannel = ServerSocketChannel.open()

View file

@ -61,7 +61,7 @@ private[spark] object Message {
if (dataBuffers.exists(_ == null)) {
throw new Exception("Attempting to create buffer message with null buffer")
}
return new BufferMessage(getNewId(), new ArrayBuffer[ByteBuffer] ++= dataBuffers, ackId)
new BufferMessage(getNewId(), new ArrayBuffer[ByteBuffer] ++= dataBuffers, ackId)
}
def createBufferMessage(dataBuffers: Seq[ByteBuffer]): BufferMessage =
@ -69,9 +69,9 @@ private[spark] object Message {
def createBufferMessage(dataBuffer: ByteBuffer, ackId: Int): BufferMessage = {
if (dataBuffer == null) {
return createBufferMessage(Array(ByteBuffer.allocate(0)), ackId)
createBufferMessage(Array(ByteBuffer.allocate(0)), ackId)
} else {
return createBufferMessage(Array(dataBuffer), ackId)
createBufferMessage(Array(dataBuffer), ackId)
}
}

View file

@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging {
resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt
val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000)
val fc = new FileClient(handler, connectTimeout)
try {

View file

@ -64,7 +64,7 @@ private[spark] object ShuffleSender {
val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
val subDir = new File(localDirs(dirId), "%02x".format(subDirId))
val file = new File(subDir, blockId.name)
return new FileSegment(file, 0, file.length())
new FileSegment(file, 0, file.length())
}
}
val sender = new ShuffleSender(port, pResovler)

View file

@ -97,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging {
throw new IOException("Checkpoint failed: temporary path " +
tempOutputPath + " already exists")
}
val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
val bufferSize = env.conf.getInt("spark.buffer.size", 65536)
val fileOutputStream = if (blockSize < 0) {
fs.create(tempOutputPath, false, bufferSize)
@ -131,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging {
): Iterator[T] = {
val env = SparkEnv.get
val fs = path.getFileSystem(broadcastedConf.value.value)
val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
val bufferSize = env.conf.getInt("spark.buffer.size", 65536)
val fileInputStream = fs.open(path, bufferSize)
val serializer = env.serializer.newInstance()
val deserializeStream = serializer.deserializeStream(fileInputStream)

View file

@ -23,8 +23,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext}
import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency}
import org.apache.spark.util.AppendOnlyMap
import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap}
private[spark] sealed trait CoGroupSplitDep extends Serializable
@ -44,14 +43,12 @@ private[spark] case class NarrowCoGroupSplitDep(
private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
private[spark]
class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
extends Partition with Serializable {
override val index: Int = idx
override def hashCode(): Int = idx
}
/**
* A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a
* tuple with the list of values for that key.
@ -62,6 +59,14 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner)
extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) {
// For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs).
// Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner.
// CoGroupValue is the intermediate state of each value before being merged in compute.
private type CoGroup = ArrayBuffer[Any]
private type CoGroupValue = (Any, Int) // Int is dependency number
private type CoGroupCombiner = Seq[CoGroup]
private val sparkConf = SparkEnv.get.conf
private var serializerClass: String = null
def setSerializer(cls: String): CoGroupedRDD[K] = {
@ -100,37 +105,74 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
override val partitioner = Some(part)
override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = {
val externalSorting = sparkConf.getBoolean("spark.shuffle.externalSorting", true)
val split = s.asInstanceOf[CoGroupPartition]
val numRdds = split.deps.size
// e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]]
val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => {
if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any])
}
val getSeq = (k: K) => {
map.changeValue(k, update)
}
val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
// A list of (rdd iterator, dependency number) pairs
val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)]
for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
// Read them from the parent
rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv =>
getSeq(kv._1)(depNum) += kv._2
}
val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]]
rddIterators += ((it, depNum))
}
case ShuffleCoGroupSplitDep(shuffleId) => {
// Read map outputs of shuffle
val fetcher = SparkEnv.get.shuffleFetcher
fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach {
kv => getSeq(kv._1)(depNum) += kv._2
}
val ser = SparkEnv.get.serializerManager.get(serializerClass, sparkConf)
val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser)
rddIterators += ((it, depNum))
}
}
new InterruptibleIterator(context, map.iterator)
if (!externalSorting) {
val map = new AppendOnlyMap[K, CoGroupCombiner]
val update: (Boolean, CoGroupCombiner) => CoGroupCombiner = (hadVal, oldVal) => {
if (hadVal) oldVal else Array.fill(numRdds)(new CoGroup)
}
val getCombiner: K => CoGroupCombiner = key => {
map.changeValue(key, update)
}
rddIterators.foreach { case (it, depNum) =>
while (it.hasNext) {
val kv = it.next()
getCombiner(kv._1)(depNum) += kv._2
}
}
new InterruptibleIterator(context, map.iterator)
} else {
val map = createExternalMap(numRdds)
rddIterators.foreach { case (it, depNum) =>
while (it.hasNext) {
val kv = it.next()
map.insert(kv._1, new CoGroupValue(kv._2, depNum))
}
}
new InterruptibleIterator(context, map.iterator)
}
}
private def createExternalMap(numRdds: Int)
: ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = {
val createCombiner: (CoGroupValue => CoGroupCombiner) = value => {
val newCombiner = Array.fill(numRdds)(new CoGroup)
value match { case (v, depNum) => newCombiner(depNum) += v }
newCombiner
}
val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner =
(combiner, value) => {
value match { case (v, depNum) => combiner(depNum) += v }
combiner
}
val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner =
(combiner1, combiner2) => {
combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 }
}
new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner](
createCombiner, mergeValue, mergeCombiners)
}
override def clearDependencies() {

View file

@ -295,10 +295,10 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc
val prefPartActual = prefPart.get
if (minPowerOfTwo.size + slack <= prefPartActual.size) // more imbalance than the slack allows
return minPowerOfTwo // prefer balance over locality
else {
return prefPartActual // prefer locality over balance
if (minPowerOfTwo.size + slack <= prefPartActual.size) { // more imbalance than the slack allows
minPowerOfTwo // prefer balance over locality
} else {
prefPartActual // prefer locality over balance
}
}
@ -331,7 +331,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc
*/
def run(): Array[PartitionGroup] = {
setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins)
throwBalls() // assign partitions (balls) to each group (bins)
throwBalls() // assign partitions (balls) to each group (bins)
getPartitions
}
}

View file

@ -19,7 +19,10 @@ package org.apache.spark.rdd
import java.io.EOFException
import org.apache.hadoop.mapred.FileInputFormat
import scala.reflect.ClassTag
import org.apache.hadoop.conf.{Configuration, Configurable}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.InputFormat
import org.apache.hadoop.mapred.InputSplit
import org.apache.hadoop.mapred.JobConf
@ -31,7 +34,7 @@ import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.NextIterator
import org.apache.hadoop.conf.{Configuration, Configurable}
import org.apache.spark.util.Utils.cloneWritables
/**
@ -42,14 +45,14 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
val inputSplit = new SerializableWritable[InputSplit](s)
override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
override def hashCode(): Int = 41 * (41 + rddId) + idx
override val index: Int = idx
}
/**
* An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
* sources in HBase, or S3).
* sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`).
*
* @param sc The SparkContext to associate the RDD with.
* @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed
@ -61,15 +64,21 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
* @param keyClass Class of the key associated with the inputFormatClass.
* @param valueClass Class of the value associated with the inputFormatClass.
* @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate.
* @param cloneRecords If true, Spark will clone the records produced by Hadoop RecordReader.
* Most RecordReader implementations reuse wrapper objects across multiple
* records, and can cause problems in RDD collect or aggregation operations.
* By default the records are cloned in Spark. However, application
* programmers can explicitly disable the cloning for better performance.
*/
class HadoopRDD[K, V](
class HadoopRDD[K: ClassTag, V: ClassTag](
sc: SparkContext,
broadcastedConf: Broadcast[SerializableWritable[Configuration]],
initLocalJobConfFuncOpt: Option[JobConf => Unit],
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int)
minSplits: Int,
cloneRecords: Boolean)
extends RDD[(K, V)](sc, Nil) with Logging {
def this(
@ -78,7 +87,8 @@ class HadoopRDD[K, V](
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int) = {
minSplits: Int,
cloneRecords: Boolean) = {
this(
sc,
sc.broadcast(new SerializableWritable(conf))
@ -87,7 +97,8 @@ class HadoopRDD[K, V](
inputFormatClass,
keyClass,
valueClass,
minSplits)
minSplits,
cloneRecords)
}
protected val jobConfCacheKey = "rdd_%d_job_conf".format(id)
@ -99,11 +110,11 @@ class HadoopRDD[K, V](
val conf: Configuration = broadcastedConf.value.value
if (conf.isInstanceOf[JobConf]) {
// A user-broadcasted JobConf was provided to the HadoopRDD, so always use it.
return conf.asInstanceOf[JobConf]
conf.asInstanceOf[JobConf]
} else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) {
// getJobConf() has been called previously, so there is already a local cache of the JobConf
// needed by this RDD.
return HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf]
HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf]
} else {
// Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the
// local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
@ -111,7 +122,7 @@ class HadoopRDD[K, V](
val newJobConf = new JobConf(broadcastedConf.value.value)
initLocalJobConfFuncOpt.map(f => f(newJobConf))
HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
return newJobConf
newJobConf
}
}
@ -127,7 +138,7 @@ class HadoopRDD[K, V](
newInputFormat.asInstanceOf[Configurable].setConf(conf)
}
HadoopRDD.putCachedMetadata(inputFormatCacheKey, newInputFormat)
return newInputFormat
newInputFormat
}
override def getPartitions: Array[Partition] = {
@ -158,10 +169,10 @@ class HadoopRDD[K, V](
// Register an on-task-completion callback to close the input stream.
context.addOnCompleteCallback{ () => closeIfNeeded() }
val key: K = reader.createKey()
val keyCloneFunc = cloneWritables[K](jobConf)
val value: V = reader.createValue()
val valueCloneFunc = cloneWritables[V](jobConf)
override def getNext() = {
try {
finished = !reader.next(key, value)
@ -169,7 +180,11 @@ class HadoopRDD[K, V](
case eof: EOFException =>
finished = true
}
(key, value)
if (cloneRecords) {
(keyCloneFunc(key.asInstanceOf[Writable]), valueCloneFunc(value.asInstanceOf[Writable]))
} else {
(key, value)
}
}
override def close() {

View file

@ -20,11 +20,14 @@ package org.apache.spark.rdd
import java.text.SimpleDateFormat
import java.util.Date
import scala.reflect.ClassTag
import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapreduce._
import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext}
import org.apache.spark.util.Utils.cloneWritables
private[spark]
@ -33,15 +36,31 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS
val serializableHadoopSplit = new SerializableWritable(rawSplit)
override def hashCode(): Int = (41 * (41 + rddId) + index)
override def hashCode(): Int = 41 * (41 + rddId) + index
}
class NewHadoopRDD[K, V](
/**
* An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
* sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`).
*
* @param sc The SparkContext to associate the RDD with.
* @param inputFormatClass Storage format of the data to be read.
* @param keyClass Class of the key associated with the inputFormatClass.
* @param valueClass Class of the value associated with the inputFormatClass.
* @param conf The Hadoop configuration.
* @param cloneRecords If true, Spark will clone the records produced by Hadoop RecordReader.
* Most RecordReader implementations reuse wrapper objects across multiple
* records, and can cause problems in RDD collect or aggregation operations.
* By default the records are cloned in Spark. However, application
* programmers can explicitly disable the cloning for better performance.
*/
class NewHadoopRDD[K: ClassTag, V: ClassTag](
sc : SparkContext,
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
@transient conf: Configuration)
@transient conf: Configuration,
cloneRecords: Boolean)
extends RDD[(K, V)](sc, Nil)
with SparkHadoopMapReduceUtil
with Logging {
@ -88,7 +107,8 @@ class NewHadoopRDD[K, V](
// Register an on-task-completion callback to close the input stream.
context.addOnCompleteCallback(() => close())
val keyCloneFunc = cloneWritables[K](conf)
val valueCloneFunc = cloneWritables[V](conf)
var havePair = false
var finished = false
@ -105,7 +125,13 @@ class NewHadoopRDD[K, V](
throw new java.util.NoSuchElementException("End of stream")
}
havePair = false
(reader.getCurrentKey, reader.getCurrentValue)
val key = reader.getCurrentKey
val value = reader.getCurrentValue
if (cloneRecords) {
(keyCloneFunc(key.asInstanceOf[Writable]), valueCloneFunc(value.asInstanceOf[Writable]))
} else {
(key, value)
}
}
private def close() {

View file

@ -18,35 +18,34 @@
package org.apache.spark.rdd
import java.nio.ByteBuffer
import java.util.Date
import java.text.SimpleDateFormat
import java.util.Date
import java.util.{HashMap => JHashMap}
import scala.collection.{mutable, Map}
import scala.collection.Map
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._
import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.mapred._
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.mapred.FileOutputFormat
import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil
import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob}
import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter}
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
import com.clearspring.analytics.stream.cardinality.HyperLogLog
// SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark.
import org.apache.hadoop.mapred.SparkHadoopWriter
import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark._
import org.apache.spark.SparkContext._
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.Aggregator
import org.apache.spark.Partitioner
import org.apache.spark.Partitioner.defaultPartitioner
import org.apache.spark.util.SerializableHyperLogLog
@ -100,8 +99,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
}, preservesPartitioning = true)
} else {
// Don't apply map-side combiner.
// A sanity check to make sure mergeCombiners is not defined.
assert(mergeCombiners == null)
val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass)
values.mapPartitionsWithContext((context, iter) => {
new InterruptibleIterator(context, aggregator.combineValuesByKey(iter))
@ -120,9 +117,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
}
/**
* Merge the values for each key using an associative function and a neutral "zero value" which may
* be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
* list concatenation, 0 for addition, or 1 for multiplication.).
* Merge the values for each key using an associative function and a neutral "zero value" which
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = {
// Serialize the zero value to a byte array so that we can get a new clone of it on each key
@ -138,18 +135,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
}
/**
* Merge the values for each key using an associative function and a neutral "zero value" which may
* be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
* list concatenation, 0 for addition, or 1 for multiplication.).
* Merge the values for each key using an associative function and a neutral "zero value" which
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = {
foldByKey(zeroValue, new HashPartitioner(numPartitions))(func)
}
/**
* Merge the values for each key using an associative function and a neutral "zero value" which may
* be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
* list concatenation, 0 for addition, or 1 for multiplication.).
* Merge the values for each key using an associative function and a neutral "zero value" which
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = {
foldByKey(zeroValue, defaultPartitioner(self))(func)
@ -226,7 +223,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
}
/**
* Return approximate number of distinct values for each key in this RDD.
* Return approximate number of distinct values for each key in this RDD.
* The accuracy of approximation can be controlled through the relative standard deviation
* (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
* more accurate counts but increase the memory footprint and vise versa. HashPartitions the
@ -268,8 +265,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
// into a hash table, leading to more objects in the old gen.
def createCombiner(v: V) = ArrayBuffer(v)
def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v
def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2
val bufs = combineByKey[ArrayBuffer[V]](
createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false)
createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false)
bufs.asInstanceOf[RDD[(K, Seq[V])]]
}
@ -340,7 +338,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
* existing partitioner/parallelism level.
*/
def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C)
: RDD[(K, C)] = {
: RDD[(K, C)] = {
combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self))
}
@ -579,7 +577,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
*/
def saveAsHadoopFile[F <: OutputFormat[K, V]](
path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) {
saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec)
val runtimeClass = fm.runtimeClass
saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec)
}
/**
@ -599,7 +598,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
conf: Configuration = self.context.hadoopConfiguration) {
conf: Configuration = self.context.hadoopConfiguration)
{
val job = new NewAPIHadoopJob(conf)
job.setOutputKeyClass(keyClass)
job.setOutputValueClass(valueClass)
@ -668,7 +668,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
codec: Option[Class[_ <: CompressionCodec]] = None) {
conf.setOutputKeyClass(keyClass)
conf.setOutputValueClass(valueClass)
// conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug
// Doesn't work in Scala 2.9 due to what may be a generics bug
// TODO: Should we uncomment this for Scala 2.10?
// conf.setOutputFormat(outputFormatClass)
conf.set("mapred.output.format.class", outputFormatClass.getName)
for (c <- codec) {
conf.setCompressMapOutput(true)
@ -702,7 +704,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
throw new SparkException("Output value class not set")
}
logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")")
logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " +
valueClass.getSimpleName+ ")")
val writer = new SparkHadoopWriter(conf)
writer.preSetup()

View file

@ -96,7 +96,7 @@ class PipedRDD[T: ClassTag](
// Return an iterator that read lines from the process's stdout
val lines = Source.fromInputStream(proc.getInputStream).getLines
return new Iterator[String] {
new Iterator[String] {
def next() = lines.next()
def hasNext = {
if (lines.hasNext) {

View file

@ -23,7 +23,6 @@ import scala.collection.Map
import scala.collection.JavaConversions.mapAsScalaMap
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.reflect.{classTag, ClassTag}
import org.apache.hadoop.io.BytesWritable
@ -52,11 +51,13 @@ import org.apache.spark._
* partitioned collection of elements that can be operated on in parallel. This class contains the
* basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition,
* [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs of key-value
* pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] contains
* operations available only on RDDs of Doubles; and [[org.apache.spark.rdd.SequenceFileRDDFunctions]]
* contains operations available on RDDs that can be saved as SequenceFiles. These operations are
* automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit
* conversions when you `import org.apache.spark.SparkContext._`.
* pairs, such as `groupByKey` and `join`;
* [[org.apache.spark.rdd.DoubleRDDFunctions]] contains operations available only on RDDs of
* Doubles; and
* [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that
* can be saved as SequenceFiles.
* These operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)]
* through implicit conversions when you `import org.apache.spark.SparkContext._`.
*
* Internally, each RDD is characterized by five main properties:
*
@ -235,12 +236,9 @@ abstract class RDD[T: ClassTag](
/**
* Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing.
*/
private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = {
if (isCheckpointed) {
firstParent[T].iterator(split, context)
} else {
compute(split, context)
}
private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] =
{
if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context)
}
// Transformations (return a new RDD)
@ -268,6 +266,9 @@ abstract class RDD[T: ClassTag](
def distinct(numPartitions: Int): RDD[T] =
map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1)
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
def distinct(): RDD[T] = distinct(partitions.size)
/**
@ -280,7 +281,7 @@ abstract class RDD[T: ClassTag](
* which can avoid performing a shuffle.
*/
def repartition(numPartitions: Int): RDD[T] = {
coalesce(numPartitions, true)
coalesce(numPartitions, shuffle = true)
}
/**
@ -651,7 +652,8 @@ abstract class RDD[T: ClassTag](
}
/**
* Reduces the elements of this RDD using the specified commutative and associative binary operator.
* Reduces the elements of this RDD using the specified commutative and
* associative binary operator.
*/
def reduce(f: (T, T) => T): T = {
val cleanF = sc.clean(f)
@ -767,7 +769,7 @@ abstract class RDD[T: ClassTag](
val entry = iter.next()
m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue)
}
return m1
m1
}
val myResult = mapPartitions(countPartition).reduce(mergeMaps)
myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map
@ -845,7 +847,7 @@ abstract class RDD[T: ClassTag](
partsScanned += numPartsToTry
}
return buf.toArray
buf.toArray
}
/**
@ -958,7 +960,7 @@ abstract class RDD[T: ClassTag](
private var storageLevel: StorageLevel = StorageLevel.NONE
/** Record user function generating this RDD. */
@transient private[spark] val origin = sc.getCallSite
@transient private[spark] val origin = sc.getCallSite()
private[spark] def elementClassTag: ClassTag[T] = classTag[T]

View file

@ -106,7 +106,7 @@ class DAGScheduler(
// 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
val RESUBMIT_TIMEOUT = 50.milliseconds
val RESUBMIT_TIMEOUT = 200.milliseconds
// The time, in millis, to wake up between polls of the completion queue in order to potentially
// resubmit failed stages
@ -133,7 +133,8 @@ class DAGScheduler(
private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
private[spark] val listenerBus = new SparkListenerBus()
// An async scheduler event bus. The bus should be stopped when DAGSCheduler is stopped.
private[spark] val listenerBus = new SparkListenerBus
// Contains the locations that each RDD's partitions are cached on
private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]]
@ -196,7 +197,7 @@ class DAGScheduler(
*/
def receive = {
case event: DAGSchedulerEvent =>
logDebug("Got event of type " + event.getClass.getName)
logTrace("Got event of type " + event.getClass.getName)
/**
* All events are forwarded to `processEvent()`, so that the event processing logic can
@ -1121,5 +1122,6 @@ class DAGScheduler(
}
metadataCleaner.cancel()
taskSched.stop()
listenerBus.stop()
}
}

View file

@ -103,7 +103,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, split)
}
return retval.toSet
retval.toSet
}
// This method does not expect failures, since validate has already passed ...
@ -121,18 +121,18 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
elem => retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, elem)
)
return retval.toSet
retval.toSet
}
private def findPreferredLocations(): Set[SplitInfo] = {
logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat +
", inputFormatClazz : " + inputFormatClazz)
if (mapreduceInputFormat) {
return prefLocsFromMapreduceInputFormat()
prefLocsFromMapreduceInputFormat()
}
else {
assert(mapredInputFormat)
return prefLocsFromMapredInputFormat()
prefLocsFromMapredInputFormat()
}
}
}

View file

@ -75,12 +75,12 @@ private[spark] class Pool(
return schedulableNameToSchedulable(schedulableName)
}
for (schedulable <- schedulableQueue) {
var sched = schedulable.getSchedulableByName(schedulableName)
val sched = schedulable.getSchedulableByName(schedulableName)
if (sched != null) {
return sched
}
}
return null
null
}
override def executorLost(executorId: String, host: String) {
@ -92,7 +92,7 @@ private[spark] class Pool(
for (schedulable <- schedulableQueue) {
shouldRevive |= schedulable.checkSpeculatableTasks()
}
return shouldRevive
shouldRevive
}
override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
@ -101,7 +101,7 @@ private[spark] class Pool(
for (schedulable <- sortedSchedulableQueue) {
sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue()
}
return sortedTaskSetQueue
sortedTaskSetQueue
}
def increaseRunningTasks(taskNum: Int) {

View file

@ -37,9 +37,9 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm {
res = math.signum(stageId1 - stageId2)
}
if (res < 0) {
return true
true
} else {
return false
false
}
}
}
@ -56,7 +56,6 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm {
val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble
val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble
val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble
var res:Boolean = true
var compare:Int = 0
if (s1Needy && !s2Needy) {
@ -70,11 +69,11 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm {
}
if (compare < 0) {
return true
true
} else if (compare > 0) {
return false
false
} else {
return s1.name < s2.name
s1.name < s2.name
}
}
}

View file

@ -43,6 +43,9 @@ case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], propertie
case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult)
extends SparkListenerEvents
/** An event used in the listener to shutdown the listener daemon thread. */
private[scheduler] case object SparkListenerShutdown extends SparkListenerEvents
trait SparkListener {
/**
* Called when a stage is completed, with information on the completed stage

View file

@ -24,15 +24,17 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
import org.apache.spark.Logging
/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */
private[spark] class SparkListenerBus() extends Logging {
private val sparkListeners = new ArrayBuffer[SparkListener]() with SynchronizedBuffer[SparkListener]
private[spark] class SparkListenerBus extends Logging {
private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener]
/* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than
* an OOM exception) if it's perpetually being added to more quickly than it's being drained. */
private val EVENT_QUEUE_CAPACITY = 10000
private val EVENT_QUEUE_CAPACITY = 10000
private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents](EVENT_QUEUE_CAPACITY)
private var queueFullErrorMessageLogged = false
// Create a new daemon thread to listen for events. This thread is stopped when it receives
// a SparkListenerShutdown event, using the stop method.
new Thread("SparkListenerBus") {
setDaemon(true)
override def run() {
@ -53,6 +55,9 @@ private[spark] class SparkListenerBus() extends Logging {
sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult))
case taskEnd: SparkListenerTaskEnd =>
sparkListeners.foreach(_.onTaskEnd(taskEnd))
case SparkListenerShutdown =>
// Get out of the while loop and shutdown the daemon thread
return
case _ =>
}
}
@ -80,7 +85,7 @@ private[spark] class SparkListenerBus() extends Logging {
*/
def waitUntilEmpty(timeoutMillis: Int): Boolean = {
val finishTime = System.currentTimeMillis + timeoutMillis
while (!eventQueue.isEmpty()) {
while (!eventQueue.isEmpty) {
if (System.currentTimeMillis > finishTime) {
return false
}
@ -88,6 +93,8 @@ private[spark] class SparkListenerBus() extends Logging {
* add overhead in the general case. */
Thread.sleep(10)
}
return true
true
}
def stop(): Unit = post(SparkListenerShutdown)
}

View file

@ -96,7 +96,7 @@ private[spark] class Stage(
def newAttemptId(): Int = {
val id = nextAttemptId
nextAttemptId += 1
return id
id
}
val name = callSite.getOrElse(rdd.origin)

View file

@ -74,6 +74,6 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long
def value(): T = {
val resultSer = SparkEnv.get.serializer.newInstance()
return resultSer.deserialize(valueBytes)
resultSer.deserialize(valueBytes)
}
}

View file

@ -31,13 +31,13 @@ import org.apache.spark.util.Utils
private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
extends Logging {
private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt
private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4)
private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool(
THREADS, "Result resolver thread")
protected val serializer = new ThreadLocal[SerializerInstance] {
override def initialValue(): SerializerInstance = {
return sparkEnv.closureSerializer.newInstance()
sparkEnv.closureSerializer.newInstance()
}
}

View file

@ -51,15 +51,15 @@ private[spark] class TaskSchedulerImpl(
isLocal: Boolean = false)
extends TaskScheduler with Logging
{
def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt)
def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4))
val conf = sc.conf
// How often to check for speculative tasks
val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong
val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100)
// Threshold above which we warn user initial TaskSet may be starved
val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong
val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000)
// TaskSetManagers are not thread safe, so any access to one should be synchronized
// on this class.
@ -125,7 +125,7 @@ private[spark] class TaskSchedulerImpl(
override def start() {
backend.start()
if (!isLocal && conf.get("spark.speculation", "false").toBoolean) {
if (!isLocal && conf.getBoolean("spark.speculation", false)) {
logInfo("Starting speculative execution thread")
import sc.env.actorSystem.dispatcher
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,

View file

@ -57,11 +57,11 @@ private[spark] class TaskSetManager(
val conf = sched.sc.conf
// CPUs to request per task
val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt
val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1)
// Quantile of tasks at which to start speculation
val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble
val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble
val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75)
val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5)
// Serializer for closures and tasks.
val env = SparkEnv.get
@ -116,7 +116,7 @@ private[spark] class TaskSetManager(
// How frequently to reprint duplicate exceptions in full, in milliseconds
val EXCEPTION_PRINT_INTERVAL =
conf.get("spark.logging.exceptionPrintInterval", "10000").toLong
conf.getLong("spark.logging.exceptionPrintInterval", 10000)
// Map of recent exceptions (identified by string representation and top stack frame) to
// duplicate count (how many times the same exception has appeared) and time the full exception
@ -228,7 +228,7 @@ private[spark] class TaskSetManager(
return Some(index)
}
}
return None
None
}
/** Check whether a task is currently running an attempt on a given host */
@ -291,7 +291,7 @@ private[spark] class TaskSetManager(
}
}
return None
None
}
/**
@ -332,7 +332,7 @@ private[spark] class TaskSetManager(
}
// Finally, if all else has failed, find a speculative task
return findSpeculativeTask(execId, host, locality)
findSpeculativeTask(execId, host, locality)
}
/**
@ -387,7 +387,7 @@ private[spark] class TaskSetManager(
case _ =>
}
}
return None
None
}
/**
@ -584,7 +584,7 @@ private[spark] class TaskSetManager(
}
override def getSchedulableByName(name: String): Schedulable = {
return null
null
}
override def addSchedulable(schedulable: Schedulable) {}
@ -594,7 +594,7 @@ private[spark] class TaskSetManager(
override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this)
sortedTaskSetQueue += this
return sortedTaskSetQueue
sortedTaskSetQueue
}
/** Called by TaskScheduler when an executor is lost so we can re-enqueue our tasks */
@ -669,7 +669,7 @@ private[spark] class TaskSetManager(
}
}
}
return foundTasks
foundTasks
}
private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {

View file

@ -63,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
// Periodically revive offers to allow delay scheduling to work
val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong
val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000)
import context.dispatcher
context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
}
@ -165,7 +165,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
override def start() {
val properties = new ArrayBuffer[(String, String)]
for ((key, value) <- scheduler.sc.conf.getAll) {
if (key.startsWith("spark.") && !key.equals("spark.hostPort")) {
if (key.startsWith("spark.")) {
properties += ((key, value))
}
}
@ -209,8 +209,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
}
override def defaultParallelism(): Int = {
conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse(
math.max(totalCoreCount.get(), 2))
conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2))
}
// Called by subclasses when notified of a lost worker

View file

@ -33,7 +33,7 @@ private[spark] class SimrSchedulerBackend(
val tmpPath = new Path(driverFilePath + "_tmp")
val filePath = new Path(driverFilePath)
val maxCores = conf.get("spark.simr.executor.cores", "1").toInt
val maxCores = conf.getInt("spark.simr.executor.cores", 1)
override def start() {
super.start()

View file

@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster
import scala.collection.mutable.HashMap
import org.apache.spark.{Logging, SparkContext}
import org.apache.spark.deploy.client.{Client, ClientListener}
import org.apache.spark.deploy.client.{AppClient, AppClientListener}
import org.apache.spark.deploy.{Command, ApplicationDescription}
import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl}
import org.apache.spark.util.Utils
@ -31,10 +31,10 @@ private[spark] class SparkDeploySchedulerBackend(
masters: Array[String],
appName: String)
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
with ClientListener
with AppClientListener
with Logging {
var client: Client = null
var client: AppClient = null
var stopping = false
var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
@ -47,14 +47,14 @@ private[spark] class SparkDeploySchedulerBackend(
val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
conf.get("spark.driver.host"), conf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}")
val command = Command(
"org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
val sparkHome = sc.getSparkHome().getOrElse(null)
val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
"http://" + sc.ui.appUIAddress)
client = new Client(sc.env.actorSystem, masters, appDesc, this, conf)
client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf)
client.start()
}

View file

@ -77,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend(
"Spark home is not set; set it through the spark.home system " +
"property, the SPARK_HOME environment variable or the SparkContext constructor"))
val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt
val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0)
var nextMesosTaskId = 0
@ -140,7 +140,7 @@ private[spark] class CoarseMesosSchedulerBackend(
.format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
}
return command.build()
command.build()
}
override def offerRescinded(d: SchedulerDriver, o: OfferID) {}

View file

@ -141,13 +141,13 @@ private[spark] class MesosSchedulerBackend(
// Serialize the map as an array of (String, String) pairs
execArgs = Utils.serialize(props.toArray)
}
return execArgs
execArgs
}
private def setClassLoader(): ClassLoader = {
val oldClassLoader = Thread.currentThread.getContextClassLoader
Thread.currentThread.setContextClassLoader(classLoader)
return oldClassLoader
oldClassLoader
}
private def restoreClassLoader(oldClassLoader: ClassLoader) {
@ -255,7 +255,7 @@ private[spark] class MesosSchedulerBackend(
.setType(Value.Type.SCALAR)
.setScalar(Value.Scalar.newBuilder().setValue(1).build())
.build()
return MesosTaskInfo.newBuilder()
MesosTaskInfo.newBuilder()
.setTaskId(taskId)
.setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
.setExecutor(createExecutorInfo(slaveId))
@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend(
}
// TODO: query Mesos for number of cores
override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt
override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8)
}

View file

@ -36,7 +36,7 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock}
*/
class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging {
private val bufferSize = {
conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024
}
def newKryoOutput() = new KryoOutput(bufferSize)
@ -48,7 +48,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial
// Allow disabling Kryo reference tracking if user knows their object graphs don't have loops.
// Do this before we invoke the user registrator so the user registrator can override this.
kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean)
kryo.setReferences(conf.getBoolean("spark.kryo.referenceTracking", true))
for (cls <- KryoSerializer.toRegister) kryo.register(cls)

View file

@ -327,7 +327,7 @@ object BlockFetcherIterator {
fetchRequestsSync.put(request)
}
copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt)
copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6))
logInfo("Started " + fetchRequestsSync.size + " remote gets in " +
Utils.getUsedTimeMs(startTime))

View file

@ -17,12 +17,14 @@
package org.apache.spark.storage
import java.util.UUID
/**
* Identifies a particular Block of data, usually associated with a single file.
* A Block can be uniquely identified by its filename, but each type of Block has a different
* set of keys which produce its unique name.
*
* If your BlockId should be serializable, be sure to add it to the BlockId.fromString() method.
* If your BlockId should be serializable, be sure to add it to the BlockId.apply() method.
*/
private[spark] sealed abstract class BlockId {
/** A globally unique identifier for this Block. Can be used for ser/de. */
@ -55,7 +57,8 @@ private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId {
def name = "broadcast_" + broadcastId
}
private[spark] case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId {
private[spark]
case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId {
def name = broadcastId.name + "_" + hType
}
@ -67,6 +70,11 @@ private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends B
def name = "input-" + streamId + "-" + uniqueId
}
/** Id associated with temporary data managed as blocks. Not serializable. */
private[spark] case class TempBlockId(id: UUID) extends BlockId {
def name = "temp_" + id
}
// Intended only for testing purposes
private[spark] case class TestBlockId(id: String) extends BlockId {
def name = "test_" + id

View file

@ -58,8 +58,8 @@ private[spark] class BlockManager(
// If we use Netty for shuffle, start a new Netty-based shuffle sender service.
private val nettyPort: Int = {
val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean
val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt
val useNetty = conf.getBoolean("spark.shuffle.use.netty", false)
val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0)
if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0
}
@ -72,19 +72,17 @@ private[spark] class BlockManager(
// Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory
// for receiving shuffle outputs)
val maxBytesInFlight =
conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024
// Whether to compress broadcast variables that are stored
val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean
val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true)
// Whether to compress shuffle output that are stored
val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean
val compressShuffle = conf.getBoolean("spark.shuffle.compress", true)
// Whether to compress RDD partitions that are stored serialized
val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean
val compressRdds = conf.getBoolean("spark.rdd.compress", false)
val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
val hostPort = Utils.localHostPort(conf)
val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
@ -159,7 +157,7 @@ private[spark] class BlockManager(
/**
* Reregister with the master and report all blocks to it. This will be called by the heart beat
* thread if our heartbeat to the block amnager indicates that we were not registered.
* thread if our heartbeat to the block manager indicates that we were not registered.
*
* Note that this method must be called without any BlockInfo locks held.
*/
@ -412,7 +410,7 @@ private[spark] class BlockManager(
logDebug("The value of block " + blockId + " is null")
}
logDebug("Block " + blockId + " not found")
return None
None
}
/**
@ -443,7 +441,7 @@ private[spark] class BlockManager(
: BlockFetcherIterator = {
val iter =
if (conf.get("spark.shuffle.use.netty", "false").toBoolean) {
if (conf.getBoolean("spark.shuffle.use.netty", false)) {
new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer)
} else {
new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer)
@ -469,7 +467,7 @@ private[spark] class BlockManager(
def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
: BlockObjectWriter = {
val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean
val syncWrites = conf.getBoolean("spark.shuffle.sync", false)
new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites)
}
@ -864,15 +862,15 @@ private[spark] object BlockManager extends Logging {
val ID_GENERATOR = new IdGenerator
def getMaxMemory(conf: SparkConf): Long = {
val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble
val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6)
(Runtime.getRuntime.maxMemory * memoryFraction).toLong
}
def getHeartBeatFrequency(conf: SparkConf): Long =
conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) / 4
def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean =
conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
conf.getBoolean("spark.test.disableBlockManagerHeartBeat", false)
/**
* Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that

View file

@ -30,8 +30,8 @@ import org.apache.spark.util.AkkaUtils
private[spark]
class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging {
val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt
val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt
val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3)
val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000)
val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"

View file

@ -348,14 +348,19 @@ object BlockManagerMasterActor {
if (storageLevel.isValid) {
// isValid means it is either stored in-memory or on-disk.
_blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize))
// But the memSize here indicates the data size in or dropped from memory,
// and the diskSize here indicates the data size in or dropped to disk.
// They can be both larger than 0, when a block is dropped from memory to disk.
// Therefore, a safe way to set BlockStatus is to set its info in accurate modes.
if (storageLevel.useMemory) {
_blocks.put(blockId, BlockStatus(storageLevel, memSize, 0))
_remainingMem -= memSize
logInfo("Added %s in memory on %s (size: %s, free: %s)".format(
blockId, blockManagerId.hostPort, Utils.bytesToString(memSize),
Utils.bytesToString(_remainingMem)))
}
if (storageLevel.useDisk) {
_blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize))
logInfo("Added %s on disk on %s (size: %s)".format(
blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize)))
}

View file

@ -42,15 +42,15 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage)
logDebug("Parsed as a block message array")
val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get)
return Some(new BlockMessageArray(responseMessages).toBufferMessage)
Some(new BlockMessageArray(responseMessages).toBufferMessage)
} catch {
case e: Exception => logError("Exception handling buffer message", e)
return None
None
}
}
case otherMessage: Any => {
logError("Unknown type message received: " + otherMessage)
return None
None
}
}
}
@ -61,7 +61,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel)
logDebug("Received [" + pB + "]")
putBlock(pB.id, pB.data, pB.level)
return None
None
}
case BlockMessage.TYPE_GET_BLOCK => {
val gB = new GetBlock(blockMessage.getId)
@ -70,9 +70,9 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
if (buffer == null) {
return None
}
return Some(BlockMessage.fromGotBlock(GotBlock(gB.id, buffer)))
Some(BlockMessage.fromGotBlock(GotBlock(gB.id, buffer)))
}
case _ => return None
case _ => None
}
}
@ -93,7 +93,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
}
logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs)
+ " and got buffer " + buffer)
return buffer
buffer
}
}
@ -111,7 +111,7 @@ private[spark] object BlockManagerWorker extends Logging {
val blockMessageArray = new BlockMessageArray(blockMessage)
val resultMessage = connectionManager.sendMessageReliablySync(
toConnManagerId, blockMessageArray.toBufferMessage)
return (resultMessage != None)
resultMessage != None
}
def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = {
@ -130,8 +130,8 @@ private[spark] object BlockManagerWorker extends Logging {
return blockMessage.getData
})
}
case None => logDebug("No response message received"); return null
case None => logDebug("No response message received")
}
return null
null
}
}

View file

@ -154,7 +154,7 @@ private[spark] class BlockMessage() {
println()
*/
val finishTime = System.currentTimeMillis
return Message.createBufferMessage(buffers)
Message.createBufferMessage(buffers)
}
override def toString: String = {

View file

@ -96,7 +96,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM
println()
println()
*/
return Message.createBufferMessage(buffers)
Message.createBufferMessage(buffers)
}
}

View file

@ -181,4 +181,8 @@ class DiskBlockObjectWriter(
// Only valid if called after close()
override def timeWriting() = _timeWriting
def bytesWritten: Long = {
lastValidPosition - initialPosition
}
}

View file

@ -19,7 +19,7 @@ package org.apache.spark.storage
import java.io.File
import java.text.SimpleDateFormat
import java.util.{Date, Random}
import java.util.{Date, Random, UUID}
import org.apache.spark.Logging
import org.apache.spark.executor.ExecutorExitCode
@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
extends PathResolver with Logging {
private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt
private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64)
// 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
@ -90,6 +90,15 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
def getFile(blockId: BlockId): File = getFile(blockId.name)
/** Produces a unique block id and File suitable for intermediate results. */
def createTempBlock(): (TempBlockId, File) = {
var blockId = new TempBlockId(UUID.randomUUID())
while (getFile(blockId).exists()) {
blockId = new TempBlockId(UUID.randomUUID())
}
(blockId, getFile(blockId))
}
private def createLocalDirs(): Array[File] = {
logDebug("Creating local directories at root dirs '" + rootDirs + "'")
val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss")

View file

@ -245,7 +245,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
return false
}
}
return true
true
}
override def contains(blockId: BlockId): Boolean = {

View file

@ -64,9 +64,9 @@ class ShuffleBlockManager(blockManager: BlockManager) {
// Turning off shuffle file consolidation causes all shuffle Blocks to get their own file.
// TODO: Remove this once the shuffle file consolidation feature is stable.
val consolidateShuffleFiles =
conf.get("spark.shuffle.consolidateFiles", "false").toBoolean
conf.getBoolean("spark.shuffle.consolidateFiles", false)
private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024
private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024
/**
* Contains all the state related to a particular shuffle. This includes a pool of unused

View file

@ -74,7 +74,7 @@ class StorageLevel private(
if (deserialized_) {
ret |= 1
}
return ret
ret
}
override def writeExternal(out: ObjectOutput) {

View file

@ -33,7 +33,7 @@ import org.apache.spark.scheduler._
*/
private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener {
// How many stages to remember
val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt
val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000)
val DEFAULT_POOL_NAME = "default"
val stageIdToPool = new HashMap[Int, String]()

View file

@ -171,7 +171,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
summary ++
<h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
<div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
<h4>Aggregated Metrics by Executors</h4> ++ executorTable.toNodeSeq() ++
<h4>Aggregated Metrics by Executor</h4> ++ executorTable.toNodeSeq() ++
<h4>Tasks</h4> ++ taskTable
headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages)

View file

@ -48,7 +48,7 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr
{if (isFairScheduler) {<th>Pool Name</th>} else {}}
<th>Description</th>
<th>Submitted</th>
<th>Task Time</th>
<th>Duration</th>
<th>Tasks: Succeeded/Total</th>
<th>Shuffle Read</th>
<th>Shuffle Write</th>

View file

@ -44,13 +44,13 @@ private[spark] object AkkaUtils {
def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false,
conf: SparkConf): (ActorSystem, Int) = {
val akkaThreads = conf.get("spark.akka.threads", "4").toInt
val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt
val akkaThreads = conf.getInt("spark.akka.threads", 4)
val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15)
val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt
val akkaTimeout = conf.getInt("spark.akka.timeout", 100)
val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt
val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean
val akkaFrameSize = conf.getInt("spark.akka.frameSize", 10)
val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false)
val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off"
if (!akkaLogLifecycleEvents) {
// As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent.
@ -58,12 +58,12 @@ private[spark] object AkkaUtils {
Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL))
}
val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off"
val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off"
val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt
val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 600)
val akkaFailureDetector =
conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble
val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt
conf.getDouble("spark.akka.failure-detector.threshold", 300.0)
val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000)
val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback(
ConfigFactory.parseString(
@ -103,7 +103,7 @@ private[spark] object AkkaUtils {
/** Returns the default Spark timeout to use for Akka ask operations. */
def askTimeout(conf: SparkConf): FiniteDuration = {
Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds")
Duration.create(conf.getLong("spark.akka.askTimeout", 30), "seconds")
}
/** Returns the default Spark timeout to use for Akka remote actor lookup. */

View file

@ -61,7 +61,7 @@ private[spark] object ClosureCleaner extends Logging {
return f.getType :: Nil // Stop at the first $outer that is not a closure
}
}
return Nil
Nil
}
// Get a list of the outer objects for a given closure object.
@ -74,7 +74,7 @@ private[spark] object ClosureCleaner extends Logging {
return f.get(obj) :: Nil // Stop at the first $outer that is not a closure
}
}
return Nil
Nil
}
private def getInnerClasses(obj: AnyRef): List[Class[_]] = {
@ -174,7 +174,7 @@ private[spark] object ClosureCleaner extends Logging {
field.setAccessible(true)
field.set(obj, outer)
}
return obj
obj
}
}
}
@ -182,7 +182,7 @@ private[spark] object ClosureCleaner extends Logging {
private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) {
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
return new MethodVisitor(ASM4) {
new MethodVisitor(ASM4) {
override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) {
if (op == GETFIELD) {
for (cl <- output.keys if cl.getName == owner.replace('/', '.')) {
@ -215,7 +215,7 @@ private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisi
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
return new MethodVisitor(ASM4) {
new MethodVisitor(ASM4) {
override def visitMethodInsn(op: Int, owner: String, name: String,
desc: String) {
val argTypes = Type.getArgumentTypes(desc)

View file

@ -74,7 +74,7 @@ object MetadataCleanerType extends Enumeration {
// initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
object MetadataCleaner {
def getDelaySeconds(conf: SparkConf) = {
conf.get("spark.cleaner.ttl", "3500").toInt
conf.getInt("spark.cleaner.ttl", -1)
}
def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int =

Some files were not shown because too many files have changed in this diff Show more