").append(m.parseHTML(a)).find(d):a)}).complete(c&&function(a,b){g.each(c,e||[a.responseText,b,a])}),this},m.expr.filters.animated=function(a){return m.grep(m.timers,function(b){return a===b.elem}).length};var cd=a.document.documentElement;function dd(a){return m.isWindow(a)?a:9===a.nodeType?a.defaultView||a.parentWindow:!1}m.offset={setOffset:function(a,b,c){var d,e,f,g,h,i,j,k=m.css(a,"position"),l=m(a),n={};"static"===k&&(a.style.position="relative"),h=l.offset(),f=m.css(a,"top"),i=m.css(a,"left"),j=("absolute"===k||"fixed"===k)&&m.inArray("auto",[f,i])>-1,j?(d=l.position(),g=d.top,e=d.left):(g=parseFloat(f)||0,e=parseFloat(i)||0),m.isFunction(b)&&(b=b.call(a,c,h)),null!=b.top&&(n.top=b.top-h.top+g),null!=b.left&&(n.left=b.left-h.left+e),"using"in b?b.using.call(a,n):l.css(n)}},m.fn.extend({offset:function(a){if(arguments.length)return void 0===a?this:this.each(function(b){m.offset.setOffset(this,a,b)});var b,c,d={top:0,left:0},e=this[0],f=e&&e.ownerDocument;if(f)return b=f.documentElement,m.contains(b,e)?(typeof e.getBoundingClientRect!==K&&(d=e.getBoundingClientRect()),c=dd(f),{top:d.top+(c.pageYOffset||b.scrollTop)-(b.clientTop||0),left:d.left+(c.pageXOffset||b.scrollLeft)-(b.clientLeft||0)}):d},position:function(){if(this[0]){var a,b,c={top:0,left:0},d=this[0];return"fixed"===m.css(d,"position")?b=d.getBoundingClientRect():(a=this.offsetParent(),b=this.offset(),m.nodeName(a[0],"html")||(c=a.offset()),c.top+=m.css(a[0],"borderTopWidth",!0),c.left+=m.css(a[0],"borderLeftWidth",!0)),{top:b.top-c.top-m.css(d,"marginTop",!0),left:b.left-c.left-m.css(d,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var a=this.offsetParent||cd;while(a&&!m.nodeName(a,"html")&&"static"===m.css(a,"position"))a=a.offsetParent;return a||cd})}}),m.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(a,b){var c=/Y/.test(b);m.fn[a]=function(d){return V(this,function(a,d,e){var f=dd(a);return void 0===e?f?b in f?f[b]:f.document.documentElement[d]:a[d]:void(f?f.scrollTo(c?m(f).scrollLeft():e,c?e:m(f).scrollTop()):a[d]=e)},a,d,arguments.length,null)}}),m.each(["top","left"],function(a,b){m.cssHooks[b]=Lb(k.pixelPosition,function(a,c){return c?(c=Jb(a,b),Hb.test(c)?m(a).position()[b]+"px":c):void 0})}),m.each({Height:"height",Width:"width"},function(a,b){m.each({padding:"inner"+a,content:b,"":"outer"+a},function(c,d){m.fn[d]=function(d,e){var f=arguments.length&&(c||"boolean"!=typeof d),g=c||(d===!0||e===!0?"margin":"border");return V(this,function(b,c,d){var e;return m.isWindow(b)?b.document.documentElement["client"+a]:9===b.nodeType?(e=b.documentElement,Math.max(b.body["scroll"+a],e["scroll"+a],b.body["offset"+a],e["offset"+a],e["client"+a])):void 0===d?m.css(b,c,g):m.style(b,c,d,g)},b,f?d:void 0,f,null)}})}),m.fn.size=function(){return this.length},m.fn.andSelf=m.fn.addBack,"function"==typeof define&&define.amd&&define("jquery",[],function(){return m});var ed=a.jQuery,fd=a.$;return m.noConflict=function(b){return a.$===m&&(a.$=fd),b&&a.jQuery===m&&(a.jQuery=ed),m},typeof b===K&&(a.jQuery=a.$=m),m});
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 11fd956bfbe66..445110d63e184 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/webui.css
+++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css
@@ -81,7 +81,9 @@ table.sortable thead {
span.kill-link {
margin-right: 2px;
+ margin-left: 20px;
color: gray;
+ float: right;
}
span.kill-link a {
@@ -112,3 +114,8 @@ pre {
padding-bottom: 0;
border: none;
}
+
+.tooltip {
+ font-weight: normal;
+}
+
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 59fdf659c9e11..79c9c451d273d 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -55,22 +55,24 @@ case class Aggregator[K, V, C] (
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.insertAll(iter)
+ // Update task metrics if context is not null
+ // TODO: Make context non optional in a future release
+ Option(context).foreach { c =>
+ c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled
+ c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled
}
- // TODO: Make this non optional in a future release
- Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled)
- Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled)
combiners.iterator
}
}
@deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0")
- def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] =
+ def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]]) : Iterator[(K, C)] =
combineCombinersByKey(iter, null)
- def combineCombinersByKey(iter: Iterator[(K, C)], context: TaskContext) : Iterator[(K, C)] = {
+ def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext)
+ : Iterator[(K, C)] =
+ {
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
var kc: Product2[K, C] = null
@@ -85,12 +87,15 @@ case class Aggregator[K, V, C] (
} else {
val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners)
while (iter.hasNext) {
- val (k, c) = iter.next()
- combiners.insert(k, c)
+ val pair = iter.next()
+ combiners.insert(pair._1, pair._2)
+ }
+ // Update task metrics if context is not null
+ // TODO: Make context non-optional in a future release
+ Option(context).foreach { c =>
+ c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled
+ c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled
}
- // TODO: Make this non optional in a future release
- Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled)
- Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled)
combiners.iterator
}
}
diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala
index 8f867686a0443..5ddda4d6953fa 100644
--- a/core/src/main/scala/org/apache/spark/CacheManager.scala
+++ b/core/src/main/scala/org/apache/spark/CacheManager.scala
@@ -17,9 +17,9 @@
package org.apache.spark
-import scala.collection.mutable.{ArrayBuffer, HashSet}
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.executor.InputMetrics
import org.apache.spark.rdd.RDD
import org.apache.spark.storage._
@@ -30,7 +30,7 @@ import org.apache.spark.storage._
private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
/** Keys of RDD partitions that are being computed/loaded. */
- private val loading = new HashSet[RDDBlockId]()
+ private val loading = new mutable.HashSet[RDDBlockId]
/** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */
def getOrCompute[T](
@@ -118,21 +118,29 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
}
/**
- * Cache the values of a partition, keeping track of any updates in the storage statuses
- * of other blocks along the way.
+ * Cache the values of a partition, keeping track of any updates in the storage statuses of
+ * other blocks along the way.
+ *
+ * The effective storage level refers to the level that actually specifies BlockManager put
+ * behavior, not the level originally specified by the user. This is mainly for forcing a
+ * MEMORY_AND_DISK partition to disk if there is not enough room to unroll the partition,
+ * while preserving the the original semantics of the RDD as specified by the application.
*/
private def putInBlockManager[T](
key: BlockId,
values: Iterator[T],
- storageLevel: StorageLevel,
- updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = {
-
- if (!storageLevel.useMemory) {
- /* This RDD is not to be cached in memory, so we can just pass the computed values
- * as an iterator directly to the BlockManager, rather than first fully unrolling
- * it in memory. The latter option potentially uses much more memory and risks OOM
- * exceptions that can be avoided. */
- updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true)
+ level: StorageLevel,
+ updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)],
+ effectiveStorageLevel: Option[StorageLevel] = None): Iterator[T] = {
+
+ val putLevel = effectiveStorageLevel.getOrElse(level)
+ if (!putLevel.useMemory) {
+ /*
+ * This RDD is not to be cached in memory, so we can just pass the computed values as an
+ * iterator directly to the BlockManager rather than first fully unrolling it in memory.
+ */
+ updatedBlocks ++=
+ blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel)
blockManager.get(key) match {
case Some(v) => v.data.asInstanceOf[Iterator[T]]
case None =>
@@ -140,14 +148,36 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
throw new BlockException(key, s"Block manager failed to return cached value for $key!")
}
} else {
- /* This RDD is to be cached in memory. In this case we cannot pass the computed values
+ /*
+ * This RDD is to be cached in memory. In this case we cannot pass the computed values
* to the BlockManager as an iterator and expect to read it back later. This is because
- * we may end up dropping a partition from memory store before getting it back, e.g.
- * when the entirety of the RDD does not fit in memory. */
- val elements = new ArrayBuffer[Any]
- elements ++= values
- updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true)
- elements.iterator.asInstanceOf[Iterator[T]]
+ * we may end up dropping a partition from memory store before getting it back.
+ *
+ * In addition, we must be careful to not unroll the entire partition in memory at once.
+ * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this
+ * single partition. Instead, we unroll the values cautiously, potentially aborting and
+ * dropping the partition to disk if applicable.
+ */
+ blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match {
+ case Left(arr) =>
+ // We have successfully unrolled the entire partition, so cache it in memory
+ updatedBlocks ++=
+ blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel)
+ arr.iterator.asInstanceOf[Iterator[T]]
+ case Right(it) =>
+ // There is not enough space to cache this partition in memory
+ logWarning(s"Not enough space to cache partition $key in memory! " +
+ s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.")
+ val returnValues = it.asInstanceOf[Iterator[T]]
+ if (putLevel.useDisk) {
+ logWarning(s"Persisting partition $key to disk instead.")
+ val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false,
+ useOffHeap = false, deserialized = false, putLevel.replication)
+ putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel))
+ } else {
+ returnValues
+ }
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index 09a60571238ea..ab2594cfc02eb 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -27,29 +27,35 @@ import org.apache.spark.shuffle.ShuffleHandle
* Base class for dependencies.
*/
@DeveloperApi
-abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
+abstract class Dependency[T] extends Serializable {
+ def rdd: RDD[T]
+}
/**
* :: DeveloperApi ::
- * Base class for dependencies where each partition of the parent RDD is used by at most one
- * partition of the child RDD. Narrow dependencies allow for pipelined execution.
+ * Base class for dependencies where each partition of the child RDD depends on a small number
+ * of partitions of the parent RDD. Narrow dependencies allow for pipelined execution.
*/
@DeveloperApi
-abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
+abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] {
/**
* Get the parent partitions for a child partition.
* @param partitionId a partition of the child RDD
* @return the partitions of the parent RDD that the child partition depends upon
*/
def getParents(partitionId: Int): Seq[Int]
+
+ override def rdd: RDD[T] = _rdd
}
/**
* :: DeveloperApi ::
- * Represents a dependency on the output of a shuffle stage.
- * @param rdd the parent RDD
+ * Represents a dependency on the output of a shuffle stage. Note that in the case of shuffle,
+ * the RDD is transient since we don't need it on the executor side.
+ *
+ * @param _rdd the parent RDD
* @param partitioner partitioner used to partition the shuffle output
* @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None,
* the default serializer, as specified by `spark.serializer` config option, will
@@ -57,20 +63,22 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
*/
@DeveloperApi
class ShuffleDependency[K, V, C](
- @transient rdd: RDD[_ <: Product2[K, V]],
+ @transient _rdd: RDD[_ <: Product2[K, V]],
val partitioner: Partitioner,
val serializer: Option[Serializer] = None,
val keyOrdering: Option[Ordering[K]] = None,
val aggregator: Option[Aggregator[K, V, C]] = None,
val mapSideCombine: Boolean = false)
- extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) {
+ extends Dependency[Product2[K, V]] {
+
+ override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]]
- val shuffleId: Int = rdd.context.newShuffleId()
+ val shuffleId: Int = _rdd.context.newShuffleId()
- val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle(
- shuffleId, rdd.partitions.size, this)
+ val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle(
+ shuffleId, _rdd.partitions.size, this)
- rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this))
+ _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this))
}
diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
new file mode 100644
index 0000000000000..24ccce21b62ca
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
@@ -0,0 +1,46 @@
+/*
+ * 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
+
+import akka.actor.Actor
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.scheduler.TaskScheduler
+
+/**
+ * A heartbeat from executors to the driver. This is a shared message used by several internal
+ * components to convey liveness or execution information for in-progress tasks.
+ */
+private[spark] case class Heartbeat(
+ executorId: String,
+ taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics
+ blockManagerId: BlockManagerId)
+
+private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean)
+
+/**
+ * Lives in the driver to receive heartbeats from executors..
+ */
+private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) extends Actor {
+ override def receive = {
+ case Heartbeat(executorId, taskMetrics, blockManagerId) =>
+ val response = HeartbeatResponse(
+ !scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId))
+ sender ! response
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 50d8e93e1f0d7..d4f2624061e35 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -39,16 +39,17 @@ trait Logging {
// be serialized and used on another machine
@transient private var log_ : Logger = null
+ // Method to get the logger name for this object
+ protected def logName = {
+ // Ignore trailing $'s in the class names for Scala objects
+ this.getClass.getName.stripSuffix("$")
+ }
+
// Method to get or create the logger for this object
protected def log: Logger = {
if (log_ == null) {
initializeIfNecessary()
- var className = this.getClass.getName
- // Ignore trailing $'s in the class names for Scala objects
- if (className.endsWith("$")) {
- className = className.substring(0, className.length - 1)
- }
- log_ = LoggerFactory.getLogger(className)
+ log_ = LoggerFactory.getLogger(logName)
}
log_
}
@@ -110,23 +111,27 @@ trait Logging {
}
private def initializeLogging() {
- // If Log4j is being used, but is not initialized, load a default properties file
- val binder = StaticLoggerBinder.getSingleton
- val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory")
- val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
- if (!log4jInitialized && usingLog4j) {
+ // Don't use a logger in here, as this is itself occurring during initialization of a logger
+ // If Log4j 1.2 is being used, but is not initialized, load a default properties file
+ val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr
+ // This distinguishes the log4j 1.2 binding, currently
+ // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently
+ // org.apache.logging.slf4j.Log4jLoggerFactory
+ val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
+ val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
+ if (!log4j12Initialized && usingLog4j12) {
val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
case Some(url) =>
PropertyConfigurator.configure(url)
- log.info(s"Using Spark's default log4j profile: $defaultLogProps")
+ System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
case None =>
System.err.println(s"Spark was unable to load $defaultLogProps")
}
}
Logging.initialized = true
- // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
+ // Force a call into slf4j to initialize it. Avoids this happening from multiple threads
// and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
log
}
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index ec99648a8488a..37053bb6f37ad 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -19,11 +19,15 @@ package org.apache.spark
import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
-import scala.reflect.ClassTag
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.{ClassTag, classTag}
+import scala.util.hashing.byteswap32
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{PartitionPruningRDD, RDD}
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.util.{CollectionsUtils, Utils}
+import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils}
/**
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
@@ -103,26 +107,49 @@ class RangePartitioner[K : Ordering : ClassTag, V](
private var ascending: Boolean = true)
extends Partitioner {
+ // We allow partitions = 0, which happens when sorting an empty RDD under the default settings.
+ require(partitions >= 0, s"Number of partitions cannot be negative but found $partitions.")
+
private var ordering = implicitly[Ordering[K]]
// An array of upper bounds for the first (partitions - 1) partitions
private var rangeBounds: Array[K] = {
- if (partitions == 1) {
- Array()
+ if (partitions <= 1) {
+ Array.empty
} else {
- val rddSize = rdd.count()
- val maxSampleSize = partitions * 20.0
- val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0)
- val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sorted
- if (rddSample.length == 0) {
- Array()
+ // This is the sample size we need to have roughly balanced output partitions, capped at 1M.
+ val sampleSize = math.min(20.0 * partitions, 1e6)
+ // Assume the input partitions are roughly balanced and over-sample a little bit.
+ val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.size).toInt
+ val (numItems, sketched) = RangePartitioner.sketch(rdd.map(_._1), sampleSizePerPartition)
+ if (numItems == 0L) {
+ Array.empty
} else {
- val bounds = new Array[K](partitions - 1)
- for (i <- 0 until partitions - 1) {
- val index = (rddSample.length - 1) * (i + 1) / partitions
- bounds(i) = rddSample(index)
+ // If a partition contains much more than the average number of items, we re-sample from it
+ // to ensure that enough items are collected from that partition.
+ val fraction = math.min(sampleSize / math.max(numItems, 1L), 1.0)
+ val candidates = ArrayBuffer.empty[(K, Float)]
+ val imbalancedPartitions = mutable.Set.empty[Int]
+ sketched.foreach { case (idx, n, sample) =>
+ if (fraction * n > sampleSizePerPartition) {
+ imbalancedPartitions += idx
+ } else {
+ // The weight is 1 over the sampling probability.
+ val weight = (n.toDouble / sample.size).toFloat
+ for (key <- sample) {
+ candidates += ((key, weight))
+ }
+ }
+ }
+ if (imbalancedPartitions.nonEmpty) {
+ // Re-sample imbalanced partitions with the desired sampling probability.
+ val imbalanced = new PartitionPruningRDD(rdd.map(_._1), imbalancedPartitions.contains)
+ val seed = byteswap32(-rdd.id - 1)
+ val reSampled = imbalanced.sample(withReplacement = false, fraction, seed).collect()
+ val weight = (1.0 / fraction).toFloat
+ candidates ++= reSampled.map(x => (x, weight))
}
- bounds
+ RangePartitioner.determineBounds(candidates, partitions)
}
}
}
@@ -134,8 +161,8 @@ class RangePartitioner[K : Ordering : ClassTag, V](
def getPartition(key: Any): Int = {
val k = key.asInstanceOf[K]
var partition = 0
- if (rangeBounds.length < 1000) {
- // If we have less than 100 partitions naive search
+ if (rangeBounds.length <= 128) {
+ // If we have less than 128 partitions naive search
while (partition < rangeBounds.length && ordering.gt(k, rangeBounds(partition))) {
partition += 1
}
@@ -212,3 +239,67 @@ class RangePartitioner[K : Ordering : ClassTag, V](
}
}
}
+
+private[spark] object RangePartitioner {
+
+ /**
+ * Sketches the input RDD via reservoir sampling on each partition.
+ *
+ * @param rdd the input RDD to sketch
+ * @param sampleSizePerPartition max sample size per partition
+ * @return (total number of items, an array of (partitionId, number of items, sample))
+ */
+ def sketch[K:ClassTag](
+ rdd: RDD[K],
+ sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = {
+ val shift = rdd.id
+ // val classTagK = classTag[K] // to avoid serializing the entire partitioner object
+ val sketched = rdd.mapPartitionsWithIndex { (idx, iter) =>
+ val seed = byteswap32(idx ^ (shift << 16))
+ val (sample, n) = SamplingUtils.reservoirSampleAndCount(
+ iter, sampleSizePerPartition, seed)
+ Iterator((idx, n, sample))
+ }.collect()
+ val numItems = sketched.map(_._2.toLong).sum
+ (numItems, sketched)
+ }
+
+ /**
+ * Determines the bounds for range partitioning from candidates with weights indicating how many
+ * items each represents. Usually this is 1 over the probability used to sample this candidate.
+ *
+ * @param candidates unordered candidates with weights
+ * @param partitions number of partitions
+ * @return selected bounds
+ */
+ def determineBounds[K:Ordering:ClassTag](
+ candidates: ArrayBuffer[(K, Float)],
+ partitions: Int): Array[K] = {
+ val ordering = implicitly[Ordering[K]]
+ val ordered = candidates.sortBy(_._1)
+ val numCandidates = ordered.size
+ val sumWeights = ordered.map(_._2.toDouble).sum
+ val step = sumWeights / partitions
+ var cumWeight = 0.0
+ var target = step
+ val bounds = ArrayBuffer.empty[K]
+ var i = 0
+ var j = 0
+ var previousBound = Option.empty[K]
+ while ((i < numCandidates) && (j < partitions - 1)) {
+ val (key, weight) = ordered(i)
+ cumWeight += weight
+ if (cumWeight > target) {
+ // Skip duplicate values.
+ if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) {
+ bounds += key
+ target += step
+ j += 1
+ previousBound = Some(key)
+ }
+ }
+ i += 1
+ }
+ bounds.toArray
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 8ce4b91cae8ae..38700847c80f4 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -40,6 +40,8 @@ import scala.collection.mutable.HashMap
*/
class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
+ import SparkConf._
+
/** Create a SparkConf that loads defaults from system properties and the classpath */
def this() = this(true)
@@ -198,7 +200,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
*
* E.g. spark.akka.option.x.y.x = "value"
*/
- getAll.filter {case (k, v) => k.startsWith("akka.")}
+ getAll.filter { case (k, _) => isAkkaConf(k) }
/** Does the configuration contain a given parameter? */
def contains(key: String): Boolean = settings.contains(key)
@@ -292,3 +294,21 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n")
}
}
+
+private[spark] object SparkConf {
+ /**
+ * Return whether the given config is an akka config (e.g. akka.actor.provider).
+ * Note that this does not include spark-specific akka configs (e.g. spark.akka.timeout).
+ */
+ def isAkkaConf(name: String): Boolean = name.startsWith("akka.")
+
+ /**
+ * Return whether the given config should be passed to an executor on start-up.
+ *
+ * Certain akka and authentication configs are required of the executor when it connects to
+ * the scheduler, while the rest of the spark configs can be inherited from the driver later.
+ */
+ def isExecutorStartupConf(name: String): Boolean = {
+ isAkkaConf(name) || name.startsWith("spark.akka") || name.startsWith("spark.auth")
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 09736be47fa65..e132955f0f850 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -36,6 +36,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob}
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.mesos.MesosNativeLibrary
+import akka.actor.Props
import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.broadcast.Broadcast
@@ -47,7 +48,7 @@ import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend}
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.scheduler.local.LocalBackend
-import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.storage._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils}
@@ -289,7 +290,7 @@ class SparkContext(config: SparkConf) extends Logging {
value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
executorEnvs(envKey) = value
}
- Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v =>
+ Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v =>
executorEnvs("SPARK_PREPEND_CLASSES") = v
}
// The Mesos scheduler backend relies on this environment variable to set executor memory.
@@ -307,6 +308,8 @@ class SparkContext(config: SparkConf) extends Logging {
// Create and start the scheduler
private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master)
+ private val heartbeatReceiver = env.actorSystem.actorOf(
+ Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver")
@volatile private[spark] var dagScheduler: DAGScheduler = _
try {
dagScheduler = new DAGScheduler(this)
@@ -455,7 +458,7 @@ class SparkContext(config: SparkConf) extends Logging {
/** Distribute a local Scala collection to form an RDD, with one or more
* location preferences (hostnames of Spark nodes) for each object.
* Create a new partition for each collection item. */
- def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = {
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
}
@@ -859,7 +862,9 @@ class SparkContext(config: SparkConf) extends Logging {
*/
@DeveloperApi
def getRDDStorageInfo: Array[RDDInfo] = {
- StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
+ val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
+ StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
+ rddInfos.filter(_.isCached)
}
/**
@@ -1009,15 +1014,15 @@ class SparkContext(config: SparkConf) extends Logging {
val dagSchedulerCopy = dagScheduler
dagScheduler = null
if (dagSchedulerCopy != null) {
+ env.metricsSystem.report()
metadataCleaner.cancel()
+ env.actorSystem.stop(heartbeatReceiver)
cleaner.foreach(_.stop())
dagSchedulerCopy.stop()
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
SparkEnv.set(null)
- ShuffleMapTask.clearCache()
- ResultTask.clearCache()
listenerBus.stop()
eventLogger.foreach(_.stop())
logInfo("Successfully stopped SparkContext")
@@ -1056,7 +1061,7 @@ class SparkContext(config: SparkConf) extends Logging {
*/
private[spark] def getCallSite(): CallSite = {
Option(getLocalProperty("externalCallSite")) match {
- case Some(callSite) => CallSite(callSite, long = "")
+ case Some(callSite) => CallSite(callSite, longForm = "")
case None => Utils.getCallSite
}
}
@@ -1078,11 +1083,12 @@ class SparkContext(config: SparkConf) extends Logging {
}
val callSite = getCallSite
val cleanedFunc = clean(func)
- logInfo("Starting job: " + callSite.short)
+ logInfo("Starting job: " + callSite.shortForm)
val start = System.nanoTime
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
resultHandler, localProperties.get)
- logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s")
+ logInfo(
+ "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s")
rdd.doCheckpoint()
}
@@ -1163,11 +1169,12 @@ class SparkContext(config: SparkConf) extends Logging {
evaluator: ApproximateEvaluator[U, R],
timeout: Long): PartialResult[R] = {
val callSite = getCallSite
- logInfo("Starting job: " + callSite.short)
+ logInfo("Starting job: " + callSite.shortForm)
val start = System.nanoTime
val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout,
localProperties.get)
- logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s")
+ logInfo(
+ "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s")
result
}
@@ -1222,10 +1229,10 @@ class SparkContext(config: SparkConf) extends Logging {
/**
* Clean a closure to make it ready to serialized and send to tasks
* (removes unreferenced variables in $outer's, updates REPL variables)
- * If
checkSerializable is set,
clean will also proactively
- * check to see if
f is serializable and throw a
SparkException
+ * If
checkSerializable is set,
clean will also proactively
+ * check to see if
f is serializable and throw a
SparkException
* if not.
- *
+ *
* @param f the closure to clean
* @param checkSerializable whether or not to immediately check
f for serializability
* @throws
SparkException if checkSerializable is set but f is not
@@ -1470,9 +1477,9 @@ object SparkContext extends Logging {
/** Creates a task scheduler based on a given master URL. Extracted for testing. */
private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = {
// Regular expression used for local[N] and local[*] master formats
- val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r
+ val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r
// Regular expression for local[N, maxRetries], used in tests with failing tasks
- val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r
+ val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r
// Regular expression for simulating a Spark cluster of [N, cores, memory] locally
val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
// Regular expression for connecting to Spark deploy clusters
@@ -1502,8 +1509,12 @@ object SparkContext extends Logging {
scheduler
case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
+ def localCpuCount = Runtime.getRuntime.availableProcessors()
+ // local[*, M] means the number of cores on the computer with M failures
+ // local[N, M] means exactly N threads with M failures
+ val threadCount = if (threads == "*") localCpuCount else threads.toInt
val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true)
- val backend = new LocalBackend(scheduler, threads.toInt)
+ val backend = new LocalBackend(scheduler, threadCount)
scheduler.initialize(backend)
scheduler
@@ -1550,7 +1561,16 @@ object SparkContext extends Logging {
throw new SparkException("YARN mode not available ?", e)
}
}
- val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+ val backend = try {
+ val clazz =
+ Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend")
+ val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
+ cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
+ } catch {
+ case e: Exception => {
+ throw new SparkException("YARN mode not available ?", e)
+ }
+ }
scheduler.initialize(backend)
scheduler
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 2b636b085d73a..0bce531aaba3e 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -18,6 +18,7 @@
package org.apache.spark
import java.io.File
+import java.net.Socket
import scala.collection.JavaConversions._
import scala.collection.mutable
@@ -67,7 +68,7 @@ class SparkEnv (
val metricsSystem: MetricsSystem,
val conf: SparkConf) extends Logging {
- // A mapping of thread ID to amount of memory used for shuffle in bytes
+ // A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations
// All accesses should be manually synchronized
val shuffleMemoryMap = mutable.HashMap[Long, Long]()
@@ -79,7 +80,7 @@ class SparkEnv (
private[spark] def stop() {
pythonWorkers.foreach { case(key, worker) => worker.stop() }
- httpFileServer.stop()
+ Option(httpFileServer).foreach(_.stop())
mapOutputTracker.stop()
shuffleManager.stop()
broadcastManager.stop()
@@ -102,10 +103,10 @@ class SparkEnv (
}
private[spark]
- def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) {
+ def destroyPythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) {
synchronized {
val key = (pythonExec, envVars)
- pythonWorkers(key).stop()
+ pythonWorkers.get(key).foreach(_.stopWorker(worker))
}
}
}
@@ -193,13 +194,7 @@ object SparkEnv extends Logging {
logInfo("Registering " + name)
actorSystem.actorOf(Props(newActor), name = name)
} else {
- val driverHost: String = conf.get("spark.driver.host", "localhost")
- 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)
- logInfo(s"Connecting to $name: $url")
- Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout)
+ AkkaUtils.makeDriverRef(name, conf, actorSystem)
}
}
@@ -228,9 +223,15 @@ object SparkEnv extends Logging {
val cacheManager = new CacheManager(blockManager)
- val httpFileServer = new HttpFileServer(securityManager)
- httpFileServer.initialize()
- conf.set("spark.fileserver.uri", httpFileServer.serverUri)
+ val httpFileServer =
+ if (isDriver) {
+ val server = new HttpFileServer(securityManager)
+ server.initialize()
+ conf.set("spark.fileserver.uri", server.serverUri)
+ server
+ } else {
+ null
+ }
val metricsSystem = if (isDriver) {
MetricsSystem.createMetricsSystem("driver", conf, securityManager)
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index df42d679b4699..8f0c5e78416c2 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -20,6 +20,7 @@ package org.apache.spark
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.Utils
/**
* :: DeveloperApi ::
@@ -88,10 +89,7 @@ case class ExceptionFailure(
stackTrace: Array[StackTraceElement],
metrics: Option[TaskMetrics])
extends TaskFailedReason {
- override def toErrorString: String = {
- val stackTraceString = if (stackTrace == null) "null" else stackTrace.mkString("\n")
- s"$className ($description}\n$stackTraceString"
- }
+ override def toErrorString: String = Utils.exceptionString(className, description, stackTrace)
}
/**
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index 885c6829a2d72..8ca731038e528 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -92,8 +92,8 @@ private[spark] object TestUtils {
def createCompiledClass(className: String, destDir: File, value: String = ""): File = {
val compiler = ToolProvider.getSystemJavaCompiler
val sourceFile = new JavaSourceFromString(className,
- "public class " + className + " { @Override public String toString() { " +
- "return \"" + value + "\";}}")
+ "public class " + className + " implements java.io.Serializable {" +
+ " @Override public String toString() { return \"" + value + "\"; }}")
// Calling this outputs a class file in pwd. It's easier to just rename the file than
// build a custom FileManager that controls the output location.
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala
new file mode 100644
index 0000000000000..0ae0b4ec042e2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.api.java
+
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.mapred.InputSplit
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.api.java.JavaSparkContext._
+import org.apache.spark.api.java.function.{Function2 => JFunction2}
+import org.apache.spark.rdd.HadoopRDD
+
+@DeveloperApi
+class JavaHadoopRDD[K, V](rdd: HadoopRDD[K, V])
+ (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V])
+ extends JavaPairRDD[K, V](rdd) {
+
+ /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */
+ @DeveloperApi
+ def mapPartitionsWithInputSplit[R](
+ f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]],
+ preservesPartitioning: Boolean = false): JavaRDD[R] = {
+ new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)),
+ preservesPartitioning)(fakeClassTag))(fakeClassTag)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala
new file mode 100644
index 0000000000000..ec4f3964d75e0
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.api.java
+
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.mapreduce.InputSplit
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.api.java.JavaSparkContext._
+import org.apache.spark.api.java.function.{Function2 => JFunction2}
+import org.apache.spark.rdd.NewHadoopRDD
+
+@DeveloperApi
+class JavaNewHadoopRDD[K, V](rdd: NewHadoopRDD[K, V])
+ (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V])
+ extends JavaPairRDD[K, V](rdd) {
+
+ /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */
+ @DeveloperApi
+ def mapPartitionsWithInputSplit[R](
+ f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]],
+ preservesPartitioning: Boolean = false): JavaRDD[R] = {
+ new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)),
+ preservesPartitioning)(fakeClassTag))(fakeClassTag)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index 4f3081433a542..76d4193e96aea 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -17,7 +17,7 @@
package org.apache.spark.api.java
-import java.util.{Comparator, List => JList}
+import java.util.{Comparator, List => JList, Map => JMap}
import java.lang.{Iterable => JIterable}
import scala.collection.JavaConversions._
@@ -122,13 +122,80 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
*/
def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] =
sample(withReplacement, fraction, Utils.random.nextLong)
-
+
/**
* Return a sampled subset of this RDD.
*/
def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed))
+ /**
+ * Return a subset of this RDD sampled by key (via stratified sampling).
+ *
+ * Create a sample of this RDD using variable sampling rates for different keys as specified by
+ * `fractions`, a key to sampling rate map.
+ *
+ * If `exact` is set to false, create the sample via simple random sampling, with one pass
+ * over the RDD, to produce a sample of size that's approximately equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over
+ * the RDD to create a sample size that's exactly equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values.
+ */
+ def sampleByKey(withReplacement: Boolean,
+ fractions: JMap[K, Double],
+ exact: Boolean,
+ seed: Long): JavaPairRDD[K, V] =
+ new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed))
+
+ /**
+ * Return a subset of this RDD sampled by key (via stratified sampling).
+ *
+ * Create a sample of this RDD using variable sampling rates for different keys as specified by
+ * `fractions`, a key to sampling rate map.
+ *
+ * If `exact` is set to false, create the sample via simple random sampling, with one pass
+ * over the RDD, to produce a sample of size that's approximately equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over
+ * the RDD to create a sample size that's exactly equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values.
+ *
+ * Use Utils.random.nextLong as the default seed for the random number generator
+ */
+ def sampleByKey(withReplacement: Boolean,
+ fractions: JMap[K, Double],
+ exact: Boolean): JavaPairRDD[K, V] =
+ sampleByKey(withReplacement, fractions, exact, Utils.random.nextLong)
+
+ /**
+ * Return a subset of this RDD sampled by key (via stratified sampling).
+ *
+ * Create a sample of this RDD using variable sampling rates for different keys as specified by
+ * `fractions`, a key to sampling rate map.
+ *
+ * Produce a sample of size that's approximately equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via
+ * simple random sampling.
+ */
+ def sampleByKey(withReplacement: Boolean,
+ fractions: JMap[K, Double],
+ seed: Long): JavaPairRDD[K, V] =
+ sampleByKey(withReplacement, fractions, false, seed)
+
+ /**
+ * Return a subset of this RDD sampled by key (via stratified sampling).
+ *
+ * Create a sample of this RDD using variable sampling rates for different keys as specified by
+ * `fractions`, a key to sampling rate map.
+ *
+ * Produce a sample of size that's approximately equal to the sum of
+ * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via
+ * simple random sampling.
+ *
+ * Use Utils.random.nextLong as the default seed for the random number generator
+ */
+ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] =
+ sampleByKey(withReplacement, fractions, false, Utils.random.nextLong)
+
/**
* Return the union of this RDD and another one. Any identical elements will appear multiple
* times (use `.distinct()` to eliminate them).
@@ -716,6 +783,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
sortByKey(comp, ascending)
}
+ /**
+ * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
+ * `collect` or `save` on the resulting RDD will return or output an ordered list of records
+ * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
+ * order of the keys).
+ */
+ def sortByKey(ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = {
+ val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]]
+ sortByKey(comp, ascending, numPartitions)
+ }
+
/**
* Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
* `collect` or `save` on the resulting RDD will return or output an ordered list of records
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 1c83a611ea76a..e0a4815940db3 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -34,7 +34,7 @@ import org.apache.spark._
import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam}
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
/**
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
@@ -112,6 +112,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
def startTime: java.lang.Long = sc.startTime
+ /** The version of Spark on which this application is running. */
+ def version: String = sc.version
+
/** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
def defaultParallelism: java.lang.Integer = sc.defaultParallelism
@@ -132,6 +135,13 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)
}
+ /** Get an RDD that has no partitions or elements. */
+ def emptyRDD[T]: JavaRDD[T] = {
+ implicit val ctag: ClassTag[T] = fakeClassTag
+ JavaRDD.fromRDD(new EmptyRDD[T](sc))
+ }
+
+
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T](list: java.util.List[T]): JavaRDD[T] =
parallelize(list, sc.defaultParallelism)
@@ -284,7 +294,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(keyClass)
implicit val ctagV: ClassTag[V] = ClassTag(valueClass)
- new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions))
+ val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions)
+ new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
/**
@@ -304,7 +315,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(keyClass)
implicit val ctagV: ClassTag[V] = ClassTag(valueClass)
- new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass))
+ val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass)
+ new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
/** Get an RDD for a Hadoop file with an arbitrary InputFormat.
@@ -323,7 +335,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(keyClass)
implicit val ctagV: ClassTag[V] = ClassTag(valueClass)
- new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions))
+ val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)
+ new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
/** Get an RDD for a Hadoop file with an arbitrary InputFormat
@@ -341,8 +354,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(keyClass)
implicit val ctagV: ClassTag[V] = ClassTag(valueClass)
- new JavaPairRDD(sc.hadoopFile(path,
- inputFormatClass, keyClass, valueClass))
+ val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass)
+ new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
/**
@@ -362,7 +375,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
conf: Configuration): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(kClass)
implicit val ctagV: ClassTag[V] = ClassTag(vClass)
- new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf))
+ val rdd = sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf)
+ new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]])
}
/**
@@ -381,7 +395,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
vClass: Class[V]): JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(kClass)
implicit val ctagV: ClassTag[V] = ClassTag(vClass)
- new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass))
+ val rdd = sc.newAPIHadoopRDD(conf, fClass, kClass, vClass)
+ new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]])
}
/** Build the union of two or more RDDs. */
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
index adaa1ef6cf9ff..f3b05e1243045 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
@@ -17,8 +17,9 @@
package org.apache.spark.api.python
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
-import org.apache.spark.Logging
+import org.apache.spark.{Logging, SerializableWritable, SparkException}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io._
import scala.util.{Failure, Success, Try}
@@ -31,13 +32,14 @@ import org.apache.spark.annotation.Experimental
* transformation code by overriding the convert method.
*/
@Experimental
-trait Converter[T, U] extends Serializable {
+trait Converter[T, + U] extends Serializable {
def convert(obj: T): U
}
private[python] object Converter extends Logging {
- def getInstance(converterClass: Option[String]): Converter[Any, Any] = {
+ def getInstance(converterClass: Option[String],
+ defaultConverter: Converter[Any, Any]): Converter[Any, Any] = {
converterClass.map { cc =>
Try {
val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]]
@@ -49,7 +51,7 @@ private[python] object Converter extends Logging {
logError(s"Failed to load converter: $cc")
throw err
}
- }.getOrElse { new DefaultConverter }
+ }.getOrElse { defaultConverter }
}
}
@@ -57,7 +59,9 @@ private[python] object Converter extends Logging {
* A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects.
* Other objects are passed through without conversion.
*/
-private[python] class DefaultConverter extends Converter[Any, Any] {
+private[python] class WritableToJavaConverter(
+ conf: Broadcast[SerializableWritable[Configuration]],
+ batchSize: Int) extends Converter[Any, Any] {
/**
* Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or
@@ -72,17 +76,30 @@ private[python] class DefaultConverter extends Converter[Any, Any] {
case fw: FloatWritable => fw.get()
case t: Text => t.toString
case bw: BooleanWritable => bw.get()
- case byw: BytesWritable => byw.getBytes
+ case byw: BytesWritable =>
+ val bytes = new Array[Byte](byw.getLength)
+ System.arraycopy(byw.getBytes(), 0, bytes, 0, byw.getLength)
+ bytes
case n: NullWritable => null
- case aw: ArrayWritable => aw.get().map(convertWritable(_))
- case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) =>
- (convertWritable(k), convertWritable(v))
- }.toMap)
+ case aw: ArrayWritable =>
+ // Due to erasure, all arrays appear as Object[] and they get pickled to Python tuples.
+ // Since we can't determine element types for empty arrays, we will not attempt to
+ // convert to primitive arrays (which get pickled to Python arrays). Users may want
+ // write custom converters for arrays if they know the element types a priori.
+ aw.get().map(convertWritable(_))
+ case mw: MapWritable =>
+ val map = new java.util.HashMap[Any, Any]()
+ mw.foreach { case (k, v) =>
+ map.put(convertWritable(k), convertWritable(v))
+ }
+ map
+ case w: Writable =>
+ if (batchSize > 1) WritableUtils.clone(w, conf.value.value) else w
case other => other
}
}
- def convert(obj: Any): Any = {
+ override def convert(obj: Any): Any = {
obj match {
case writable: Writable =>
convertWritable(writable)
@@ -92,6 +109,47 @@ private[python] class DefaultConverter extends Converter[Any, Any] {
}
}
+/**
+ * A converter that converts common types to [[org.apache.hadoop.io.Writable]]. Note that array
+ * types are not supported since the user needs to subclass [[org.apache.hadoop.io.ArrayWritable]]
+ * to set the type properly. See [[org.apache.spark.api.python.DoubleArrayWritable]] and
+ * [[org.apache.spark.api.python.DoubleArrayToWritableConverter]] for an example. They are used in
+ * PySpark RDD `saveAsNewAPIHadoopFile` doctest.
+ */
+private[python] class JavaToWritableConverter extends Converter[Any, Writable] {
+
+ /**
+ * Converts common data types to [[org.apache.hadoop.io.Writable]]. Note that array types are not
+ * supported out-of-the-box.
+ */
+ private def convertToWritable(obj: Any): Writable = {
+ import collection.JavaConversions._
+ obj match {
+ case i: java.lang.Integer => new IntWritable(i)
+ case d: java.lang.Double => new DoubleWritable(d)
+ case l: java.lang.Long => new LongWritable(l)
+ case f: java.lang.Float => new FloatWritable(f)
+ case s: java.lang.String => new Text(s)
+ case b: java.lang.Boolean => new BooleanWritable(b)
+ case aob: Array[Byte] => new BytesWritable(aob)
+ case null => NullWritable.get()
+ case map: java.util.Map[_, _] =>
+ val mapWritable = new MapWritable()
+ map.foreach { case (k, v) =>
+ mapWritable.put(convertToWritable(k), convertToWritable(v))
+ }
+ mapWritable
+ case other => throw new SparkException(
+ s"Data of type ${other.getClass.getName} cannot be used")
+ }
+ }
+
+ override def convert(obj: Any): Writable = obj match {
+ case writable: Writable => writable
+ case other => convertToWritable(other)
+ }
+}
+
/** Utilities for working with Python objects <-> Hadoop-related objects */
private[python] object PythonHadoopUtil {
@@ -118,7 +176,7 @@ private[python] object PythonHadoopUtil {
/**
* Converts an RDD of key-value pairs, where key and/or value could be instances of
- * [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)]
+ * [[org.apache.hadoop.io.Writable]], into an RDD of base types, or vice versa.
*/
def convertRDD[K, V](rdd: RDD[(K, V)],
keyConverter: Converter[Any, Any],
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 462e09466bfa6..0b5322c6fb965 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -23,22 +23,25 @@ import java.nio.charset.Charset
import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections}
import scala.collection.JavaConversions._
+import scala.language.existentials
import scala.reflect.ClassTag
-import scala.util.Try
+import scala.util.{Try, Success, Failure}
import net.razorvine.pickle.{Pickler, Unpickler}
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapred.{InputFormat, JobConf}
-import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.hadoop.io.compress.CompressionCodec
+import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf}
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat}
import org.apache.spark._
+import org.apache.spark.SparkContext._
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.util.Utils
-private[spark] class PythonRDD[T: ClassTag](
- parent: RDD[T],
+private[spark] class PythonRDD(
+ parent: RDD[_],
command: Array[Byte],
envVars: JMap[String, String],
pythonIncludes: JList[String],
@@ -57,6 +60,9 @@ private[spark] class PythonRDD[T: ClassTag](
override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
val startTime = System.currentTimeMillis
val env = SparkEnv.get
+ val localdir = env.blockManager.diskBlockManager.localDirs.map(
+ f => f.getPath()).mkString(",")
+ envVars += ("SPARK_LOCAL_DIR" -> localdir) // it's also used in monitor thread
val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap)
// Start a thread to feed the process input from our parent's iterator
@@ -235,7 +241,7 @@ private[spark] class PythonRDD[T: ClassTag](
if (!context.completed) {
try {
logWarning("Incomplete task interrupted: Attempting to kill Python Worker")
- env.destroyPythonWorker(pythonExec, envVars.toMap)
+ env.destroyPythonWorker(pythonExec, envVars.toMap, worker)
} catch {
case e: Exception =>
logError("Exception when trying to kill worker", e)
@@ -362,19 +368,17 @@ private[spark] object PythonRDD extends Logging {
valueClassMaybeNull: String,
keyConverterClass: String,
valueConverterClass: String,
- minSplits: Int) = {
+ minSplits: Int,
+ batchSize: Int) = {
val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text")
val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text")
- implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]]
- implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]]
- val kc = kcm.runtimeClass.asInstanceOf[Class[K]]
- val vc = vcm.runtimeClass.asInstanceOf[Class[V]]
-
+ val kc = Class.forName(keyClass).asInstanceOf[Class[K]]
+ val vc = Class.forName(valueClass).asInstanceOf[Class[V]]
val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits)
- val keyConverter = Converter.getInstance(Option(keyConverterClass))
- val valueConverter = Converter.getInstance(Option(valueConverterClass))
- val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter)
- JavaRDD.fromRDD(SerDeUtil.rddToPython(converted))
+ val confBroadcasted = sc.sc.broadcast(new SerializableWritable(sc.hadoopConfiguration()))
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new WritableToJavaConverter(confBroadcasted, batchSize))
+ JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize))
}
/**
@@ -391,17 +395,16 @@ private[spark] object PythonRDD extends Logging {
valueClass: String,
keyConverterClass: String,
valueConverterClass: String,
- confAsMap: java.util.HashMap[String, String]) = {
- val conf = PythonHadoopUtil.mapToConf(confAsMap)
- val baseConf = sc.hadoopConfiguration()
- val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf)
+ confAsMap: java.util.HashMap[String, String],
+ batchSize: Int) = {
+ val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration())
val rdd =
newAPIHadoopRDDFromClassNames[K, V, F](sc,
Some(path), inputFormatClass, keyClass, valueClass, mergedConf)
- val keyConverter = Converter.getInstance(Option(keyConverterClass))
- val valueConverter = Converter.getInstance(Option(valueConverterClass))
- val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter)
- JavaRDD.fromRDD(SerDeUtil.rddToPython(converted))
+ val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf))
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new WritableToJavaConverter(confBroadcasted, batchSize))
+ JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize))
}
/**
@@ -418,15 +421,16 @@ private[spark] object PythonRDD extends Logging {
valueClass: String,
keyConverterClass: String,
valueConverterClass: String,
- confAsMap: java.util.HashMap[String, String]) = {
+ confAsMap: java.util.HashMap[String, String],
+ batchSize: Int) = {
val conf = PythonHadoopUtil.mapToConf(confAsMap)
val rdd =
newAPIHadoopRDDFromClassNames[K, V, F](sc,
None, inputFormatClass, keyClass, valueClass, conf)
- val keyConverter = Converter.getInstance(Option(keyConverterClass))
- val valueConverter = Converter.getInstance(Option(valueConverterClass))
- val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter)
- JavaRDD.fromRDD(SerDeUtil.rddToPython(converted))
+ val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf))
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new WritableToJavaConverter(confBroadcasted, batchSize))
+ JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize))
}
private def newAPIHadoopRDDFromClassNames[K, V, F <: NewInputFormat[K, V]](
@@ -436,18 +440,14 @@ private[spark] object PythonRDD extends Logging {
keyClass: String,
valueClass: String,
conf: Configuration) = {
- implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]]
- implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]]
- implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]]
- val kc = kcm.runtimeClass.asInstanceOf[Class[K]]
- val vc = vcm.runtimeClass.asInstanceOf[Class[V]]
- val fc = fcm.runtimeClass.asInstanceOf[Class[F]]
- val rdd = if (path.isDefined) {
+ val kc = Class.forName(keyClass).asInstanceOf[Class[K]]
+ val vc = Class.forName(valueClass).asInstanceOf[Class[V]]
+ val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]]
+ if (path.isDefined) {
sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf)
} else {
sc.sc.newAPIHadoopRDD[K, V, F](conf, fc, kc, vc)
}
- rdd
}
/**
@@ -464,17 +464,16 @@ private[spark] object PythonRDD extends Logging {
valueClass: String,
keyConverterClass: String,
valueConverterClass: String,
- confAsMap: java.util.HashMap[String, String]) = {
- val conf = PythonHadoopUtil.mapToConf(confAsMap)
- val baseConf = sc.hadoopConfiguration()
- val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf)
+ confAsMap: java.util.HashMap[String, String],
+ batchSize: Int) = {
+ val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration())
val rdd =
hadoopRDDFromClassNames[K, V, F](sc,
Some(path), inputFormatClass, keyClass, valueClass, mergedConf)
- val keyConverter = Converter.getInstance(Option(keyConverterClass))
- val valueConverter = Converter.getInstance(Option(valueConverterClass))
- val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter)
- JavaRDD.fromRDD(SerDeUtil.rddToPython(converted))
+ val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf))
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new WritableToJavaConverter(confBroadcasted, batchSize))
+ JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize))
}
/**
@@ -491,15 +490,16 @@ private[spark] object PythonRDD extends Logging {
valueClass: String,
keyConverterClass: String,
valueConverterClass: String,
- confAsMap: java.util.HashMap[String, String]) = {
+ confAsMap: java.util.HashMap[String, String],
+ batchSize: Int) = {
val conf = PythonHadoopUtil.mapToConf(confAsMap)
val rdd =
hadoopRDDFromClassNames[K, V, F](sc,
None, inputFormatClass, keyClass, valueClass, conf)
- val keyConverter = Converter.getInstance(Option(keyConverterClass))
- val valueConverter = Converter.getInstance(Option(valueConverterClass))
- val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter)
- JavaRDD.fromRDD(SerDeUtil.rddToPython(converted))
+ val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf))
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new WritableToJavaConverter(confBroadcasted, batchSize))
+ JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize))
}
private def hadoopRDDFromClassNames[K, V, F <: InputFormat[K, V]](
@@ -509,18 +509,14 @@ private[spark] object PythonRDD extends Logging {
keyClass: String,
valueClass: String,
conf: Configuration) = {
- implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]]
- implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]]
- implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]]
- val kc = kcm.runtimeClass.asInstanceOf[Class[K]]
- val vc = vcm.runtimeClass.asInstanceOf[Class[V]]
- val fc = fcm.runtimeClass.asInstanceOf[Class[F]]
- val rdd = if (path.isDefined) {
+ val kc = Class.forName(keyClass).asInstanceOf[Class[K]]
+ val vc = Class.forName(valueClass).asInstanceOf[Class[V]]
+ val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]]
+ if (path.isDefined) {
sc.sc.hadoopFile(path.get, fc, kc, vc)
} else {
sc.sc.hadoopRDD(new JobConf(conf), fc, kc, vc)
}
- rdd
}
def writeUTF(str: String, dataOut: DataOutputStream) {
@@ -540,24 +536,199 @@ private[spark] object PythonRDD extends Logging {
file.close()
}
+ private def getMergedConf(confAsMap: java.util.HashMap[String, String],
+ baseConf: Configuration): Configuration = {
+ val conf = PythonHadoopUtil.mapToConf(confAsMap)
+ PythonHadoopUtil.mergeConfs(baseConf, conf)
+ }
+
+ private def inferKeyValueTypes[K, V](rdd: RDD[(K, V)], keyConverterClass: String = null,
+ valueConverterClass: String = null): (Class[_], Class[_]) = {
+ // Peek at an element to figure out key/value types. Since Writables are not serializable,
+ // we cannot call first() on the converted RDD. Instead, we call first() on the original RDD
+ // and then convert locally.
+ val (key, value) = rdd.first()
+ val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass,
+ new JavaToWritableConverter)
+ (kc.convert(key).getClass, vc.convert(value).getClass)
+ }
+
+ private def getKeyValueTypes(keyClass: String, valueClass: String):
+ Option[(Class[_], Class[_])] = {
+ for {
+ k <- Option(keyClass)
+ v <- Option(valueClass)
+ } yield (Class.forName(k), Class.forName(v))
+ }
+
+ private def getKeyValueConverters(keyConverterClass: String, valueConverterClass: String,
+ defaultConverter: Converter[Any, Any]): (Converter[Any, Any], Converter[Any, Any]) = {
+ val keyConverter = Converter.getInstance(Option(keyConverterClass), defaultConverter)
+ val valueConverter = Converter.getInstance(Option(valueConverterClass), defaultConverter)
+ (keyConverter, valueConverter)
+ }
+
+ /**
+ * Convert an RDD of key-value pairs from internal types to serializable types suitable for
+ * output, or vice versa.
+ */
+ private def convertRDD[K, V](rdd: RDD[(K, V)],
+ keyConverterClass: String,
+ valueConverterClass: String,
+ defaultConverter: Converter[Any, Any]): RDD[(Any, Any)] = {
+ val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass,
+ defaultConverter)
+ PythonHadoopUtil.convertRDD(rdd, kc, vc)
+ }
+
+ /**
+ * Output a Python RDD of key-value pairs as a Hadoop SequenceFile using the Writable types
+ * we convert from the RDD's key and value types. Note that keys and values can't be
+ * [[org.apache.hadoop.io.Writable]] types already, since Writables are not Java
+ * `Serializable` and we can't peek at them. The `path` can be on any Hadoop file system.
+ */
+ def saveAsSequenceFile[K, V, C <: CompressionCodec](
+ pyRDD: JavaRDD[Array[Byte]],
+ batchSerialized: Boolean,
+ path: String,
+ compressionCodecClass: String) = {
+ saveAsHadoopFile(
+ pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat",
+ null, null, null, null, new java.util.HashMap(), compressionCodecClass)
+ }
+
+ /**
+ * Output a Python RDD of key-value pairs to any Hadoop file system, using old Hadoop
+ * `OutputFormat` in mapred package. Keys and values are converted to suitable output
+ * types using either user specified converters or, if not specified,
+ * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types
+ * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in
+ * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of
+ * this RDD.
+ */
+ def saveAsHadoopFile[K, V, F <: OutputFormat[_, _], C <: CompressionCodec](
+ pyRDD: JavaRDD[Array[Byte]],
+ batchSerialized: Boolean,
+ path: String,
+ outputFormatClass: String,
+ keyClass: String,
+ valueClass: String,
+ keyConverterClass: String,
+ valueConverterClass: String,
+ confAsMap: java.util.HashMap[String, String],
+ compressionCodecClass: String) = {
+ val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized)
+ val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse(
+ inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass))
+ val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration)
+ val codec = Option(compressionCodecClass).map(Class.forName(_).asInstanceOf[Class[C]])
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new JavaToWritableConverter)
+ val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]]
+ converted.saveAsHadoopFile(path, kc, vc, fc, new JobConf(mergedConf), codec=codec)
+ }
+
+ /**
+ * Output a Python RDD of key-value pairs to any Hadoop file system, using new Hadoop
+ * `OutputFormat` in mapreduce package. Keys and values are converted to suitable output
+ * types using either user specified converters or, if not specified,
+ * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types
+ * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in
+ * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of
+ * this RDD.
+ */
+ def saveAsNewAPIHadoopFile[K, V, F <: NewOutputFormat[_, _]](
+ pyRDD: JavaRDD[Array[Byte]],
+ batchSerialized: Boolean,
+ path: String,
+ outputFormatClass: String,
+ keyClass: String,
+ valueClass: String,
+ keyConverterClass: String,
+ valueConverterClass: String,
+ confAsMap: java.util.HashMap[String, String]) = {
+ val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized)
+ val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse(
+ inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass))
+ val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration)
+ val converted = convertRDD(rdd, keyConverterClass, valueConverterClass,
+ new JavaToWritableConverter)
+ val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]]
+ converted.saveAsNewAPIHadoopFile(path, kc, vc, fc, mergedConf)
+ }
+
/**
- * Convert an RDD of serialized Python dictionaries to Scala Maps
- * TODO: Support more Python types.
+ * Output a Python RDD of key-value pairs to any Hadoop file system, using a Hadoop conf
+ * converted from the passed-in `confAsMap`. The conf should set relevant output params (
+ * e.g., output path, output format, etc), in the same way as it would be configured for
+ * a Hadoop MapReduce job. Both old and new Hadoop OutputFormat APIs are supported
+ * (mapred vs. mapreduce). Keys/values are converted for output using either user specified
+ * converters or, by default, [[org.apache.spark.api.python.JavaToWritableConverter]].
*/
+ def saveAsHadoopDataset[K, V](
+ pyRDD: JavaRDD[Array[Byte]],
+ batchSerialized: Boolean,
+ confAsMap: java.util.HashMap[String, String],
+ keyConverterClass: String,
+ valueConverterClass: String,
+ useNewAPI: Boolean) = {
+ val conf = PythonHadoopUtil.mapToConf(confAsMap)
+ val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized),
+ keyConverterClass, valueConverterClass, new JavaToWritableConverter)
+ if (useNewAPI) {
+ converted.saveAsNewAPIHadoopDataset(conf)
+ } else {
+ converted.saveAsHadoopDataset(new JobConf(conf))
+ }
+ }
+
+
+ /**
+ * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions).
+ */
+ @deprecated("PySpark does not use it anymore", "1.1")
def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = {
pyRDD.rdd.mapPartitions { iter =>
val unpickle = new Unpickler
- // TODO: Figure out why flatMap is necessay for pyspark
iter.flatMap { row =>
unpickle.loads(row) match {
- case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap)
- // Incase the partition doesn't have a collection
+ // in case of objects are pickled in batch mode
+ case objs: JArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap)
+ // not in batch mode
case obj: JMap[String @unchecked, _] => Seq(obj.toMap)
}
}
}
}
+ /**
+ * Convert an RDD of serialized Python tuple to Array (no recursive conversions).
+ * It is only used by pyspark.sql.
+ */
+ def pythonToJavaArray(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Array[_]] = {
+
+ def toArray(obj: Any): Array[_] = {
+ obj match {
+ case objs: JArrayList[_] =>
+ objs.toArray
+ case obj if obj.getClass.isArray =>
+ obj.asInstanceOf[Array[_]].toArray
+ }
+ }
+
+ pyRDD.rdd.mapPartitions { iter =>
+ val unpickle = new Unpickler
+ iter.flatMap { row =>
+ val obj = unpickle.loads(row)
+ if (batched) {
+ obj.asInstanceOf[JArrayList[_]].map(toArray)
+ } else {
+ Seq(toArray(obj))
+ }
+ }
+ }.toJavaRDD()
+ }
+
/**
* Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by
* PySpark.
@@ -588,19 +759,30 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536)
+ /**
+ * We try to reuse a single Socket to transfer accumulator updates, as they are all added
+ * by the DAGScheduler's single-threaded actor anyway.
+ */
+ @transient var socket: Socket = _
+
+ def openSocket(): Socket = synchronized {
+ if (socket == null || socket.isClosed) {
+ socket = new Socket(serverHost, serverPort)
+ }
+ socket
+ }
+
override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]])
- : JList[Array[Byte]] = {
+ : JList[Array[Byte]] = synchronized {
if (serverHost == null) {
// This happens on the worker node, where we just want to remember all the updates
val1.addAll(val2)
val1
} else {
// This happens on the master, where we pass the updates to Python through a socket
- val socket = new Socket(serverHost, serverPort)
- // SPARK-2282: Immediately reuse closed sockets because we create one per task.
- socket.setReuseAddress(true)
+ val socket = openSocket()
val in = socket.getInputStream
val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize))
out.writeInt(val2.size)
@@ -614,7 +796,6 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
if (byteRead == -1) {
throw new SparkException("EOF reached before Python server acknowledged")
}
- socket.close()
null
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
index 6d3e257c4d5df..52c70712eea3d 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
@@ -29,7 +29,7 @@ private[spark] object PythonUtils {
val pythonPath = new ArrayBuffer[String]
for (sparkHome <- sys.env.get("SPARK_HOME")) {
pythonPath += Seq(sparkHome, "python").mkString(File.separator)
- pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator)
+ pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.2.1-src.zip").mkString(File.separator)
}
pythonPath ++= SparkContext.jarOfObject(this)
pythonPath.mkString(File.pathSeparator)
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 759cbe2c46c52..7af260d0b7f26 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -17,9 +17,11 @@
package org.apache.spark.api.python
-import java.io.{DataInputStream, InputStream, OutputStreamWriter}
+import java.lang.Runtime
+import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter}
import java.net.{InetAddress, ServerSocket, Socket, SocketException}
+import scala.collection.mutable
import scala.collection.JavaConversions._
import org.apache.spark._
@@ -39,6 +41,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
var daemon: Process = null
val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1))
var daemonPort: Int = 0
+ var daemonWorkers = new mutable.WeakHashMap[Socket, Int]()
+
+ var simpleWorkers = new mutable.WeakHashMap[Socket, Process]()
val pythonPath = PythonUtils.mergePythonPaths(
PythonUtils.sparkPythonPath,
@@ -58,19 +63,31 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
* to avoid the high cost of forking from Java. This currently only works on UNIX-based systems.
*/
private def createThroughDaemon(): Socket = {
+
+ def createSocket(): Socket = {
+ val socket = new Socket(daemonHost, daemonPort)
+ val pid = new DataInputStream(socket.getInputStream).readInt()
+ if (pid < 0) {
+ throw new IllegalStateException("Python daemon failed to launch worker")
+ }
+ daemonWorkers.put(socket, pid)
+ socket
+ }
+
synchronized {
// Start the daemon if it hasn't been started
startDaemon()
// Attempt to connect, restart and retry once if it fails
try {
- new Socket(daemonHost, daemonPort)
+ createSocket()
} catch {
case exc: SocketException =>
- logWarning("Python daemon unexpectedly quit, attempting to restart")
+ logWarning("Failed to open socket to Python daemon:", exc)
+ logWarning("Assuming that daemon unexpectedly quit, attempting to restart")
stopDaemon()
startDaemon()
- new Socket(daemonHost, daemonPort)
+ createSocket()
}
}
}
@@ -101,7 +118,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
// Wait for it to connect to our socket
serverSocket.setSoTimeout(10000)
try {
- return serverSocket.accept()
+ val socket = serverSocket.accept()
+ simpleWorkers.put(socket, worker)
+ return socket
} catch {
case e: Exception =>
throw new SparkException("Python worker did not connect back in time", e)
@@ -183,19 +202,40 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
private def stopDaemon() {
synchronized {
- // Request shutdown of existing daemon by sending SIGTERM
- if (daemon != null) {
- daemon.destroy()
+ if (useDaemon) {
+ // Request shutdown of existing daemon by sending SIGTERM
+ if (daemon != null) {
+ daemon.destroy()
+ }
+
+ daemon = null
+ daemonPort = 0
+ } else {
+ simpleWorkers.mapValues(_.destroy())
}
-
- daemon = null
- daemonPort = 0
}
}
def stop() {
stopDaemon()
}
+
+ def stopWorker(worker: Socket) {
+ if (useDaemon) {
+ if (daemon != null) {
+ daemonWorkers.get(worker).foreach { pid =>
+ // tell daemon to kill worker by pid
+ val output = new DataOutputStream(daemon.getOutputStream)
+ output.writeInt(pid)
+ output.flush()
+ daemon.getOutputStream.flush()
+ }
+ }
+ } else {
+ simpleWorkers.get(worker).foreach(_.destroy())
+ }
+ worker.close()
+ }
}
private object PythonWorkerFactory {
diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
index 9a012e7254901..efc9009c088a8 100644
--- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
@@ -17,13 +17,14 @@
package org.apache.spark.api.python
-import scala.util.Try
-import org.apache.spark.rdd.RDD
-import org.apache.spark.Logging
-import scala.util.Success
+import scala.collection.JavaConversions._
import scala.util.Failure
-import net.razorvine.pickle.Pickler
+import scala.util.Try
+import net.razorvine.pickle.{Unpickler, Pickler}
+
+import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.rdd.RDD
/** Utilities for serialization / deserialization between Python and Java, using Pickle. */
private[python] object SerDeUtil extends Logging {
@@ -65,20 +66,52 @@ private[python] object SerDeUtil extends Logging {
* by PySpark. By default, if serialization fails, toString is called and the string
* representation is serialized
*/
- def rddToPython(rdd: RDD[(Any, Any)]): RDD[Array[Byte]] = {
+ def pairRDDToPython(rdd: RDD[(Any, Any)], batchSize: Int): RDD[Array[Byte]] = {
val (keyFailed, valueFailed) = checkPickle(rdd.first())
rdd.mapPartitions { iter =>
val pickle = new Pickler
- iter.map { case (k, v) =>
- if (keyFailed && valueFailed) {
- pickle.dumps(Array(k.toString, v.toString))
- } else if (keyFailed) {
- pickle.dumps(Array(k.toString, v))
- } else if (!keyFailed && valueFailed) {
- pickle.dumps(Array(k, v.toString))
+ val cleaned = iter.map { case (k, v) =>
+ val key = if (keyFailed) k.toString else k
+ val value = if (valueFailed) v.toString else v
+ Array[Any](key, value)
+ }
+ if (batchSize > 1) {
+ cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched)))
+ } else {
+ cleaned.map(pickle.dumps(_))
+ }
+ }
+ }
+
+ /**
+ * Convert an RDD of serialized Python tuple (K, V) to RDD[(K, V)].
+ */
+ def pythonToPairRDD[K, V](pyRDD: RDD[Array[Byte]], batchSerialized: Boolean): RDD[(K, V)] = {
+ def isPair(obj: Any): Boolean = {
+ Option(obj.getClass.getComponentType).map(!_.isPrimitive).getOrElse(false) &&
+ obj.asInstanceOf[Array[_]].length == 2
+ }
+ pyRDD.mapPartitions { iter =>
+ val unpickle = new Unpickler
+ val unpickled =
+ if (batchSerialized) {
+ iter.flatMap { batch =>
+ unpickle.loads(batch) match {
+ case objs: java.util.List[_] => collectionAsScalaIterable(objs)
+ case other => throw new SparkException(
+ s"Unexpected type ${other.getClass.getName} for batch serialized Python RDD")
+ }
+ }
} else {
- pickle.dumps(Array(k, v))
+ iter.map(unpickle.loads(_))
}
+ unpickled.map {
+ case obj if isPair(obj) =>
+ // we only accept (K, V)
+ val arr = obj.asInstanceOf[Array[_]]
+ (arr.head.asInstanceOf[K], arr.last.asInstanceOf[V])
+ case other => throw new SparkException(
+ s"RDD element of type ${other.getClass.getName} cannot be used")
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
index f0e3fb9aff5a0..d11db978b842e 100644
--- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
@@ -17,15 +17,16 @@
package org.apache.spark.api.python
-import org.apache.spark.SparkContext
-import org.apache.hadoop.io._
-import scala.Array
import java.io.{DataOutput, DataInput}
+import java.nio.charset.Charset
+
+import org.apache.hadoop.io._
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat
import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.{SparkContext, SparkException}
/**
- * A class to test MsgPack serialization on the Scala side, that will be deserialized
+ * A class to test Pyrolite serialization on the Scala side, that will be deserialized
* in Python
* @param str
* @param int
@@ -54,7 +55,13 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten
}
}
-class TestConverter extends Converter[Any, Any] {
+private[python] class TestInputKeyConverter extends Converter[Any, Any] {
+ override def convert(obj: Any) = {
+ obj.asInstanceOf[IntWritable].get().toChar
+ }
+}
+
+private[python] class TestInputValueConverter extends Converter[Any, Any] {
import collection.JavaConversions._
override def convert(obj: Any) = {
val m = obj.asInstanceOf[MapWritable]
@@ -62,6 +69,38 @@ class TestConverter extends Converter[Any, Any] {
}
}
+private[python] class TestOutputKeyConverter extends Converter[Any, Any] {
+ override def convert(obj: Any) = {
+ new Text(obj.asInstanceOf[Int].toString)
+ }
+}
+
+private[python] class TestOutputValueConverter extends Converter[Any, Any] {
+ import collection.JavaConversions._
+ override def convert(obj: Any) = {
+ new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head)
+ }
+}
+
+private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable])
+
+private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] {
+ override def convert(obj: Any) = obj match {
+ case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] =>
+ val daw = new DoubleArrayWritable
+ daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_)))
+ daw
+ case other => throw new SparkException(s"Data of type $other is not supported")
+ }
+}
+
+private[python] class WritableToDoubleArrayConverter extends Converter[Any, Array[Double]] {
+ override def convert(obj: Any): Array[Double] = obj match {
+ case daw : DoubleArrayWritable => daw.get().map(_.asInstanceOf[DoubleWritable].get())
+ case other => throw new SparkException(s"Data of type $other is not supported")
+ }
+}
+
/**
* This object contains method to generate SequenceFile test data and write it to a
* given directory (probably a temp directory)
@@ -97,7 +136,8 @@ object WriteInputFormatTestDataGenerator {
sc.parallelize(intKeys).saveAsSequenceFile(intPath)
sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath)
sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath)
- sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes) }).saveAsSequenceFile(bytesPath)
+ sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(Charset.forName("UTF-8"))) }
+ ).saveAsSequenceFile(bytesPath)
val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false))
sc.parallelize(bools).saveAsSequenceFile(boolPath)
sc.parallelize(intKeys).map{ case (k, v) =>
@@ -106,19 +146,20 @@ object WriteInputFormatTestDataGenerator {
// Create test data for ArrayWritable
val data = Seq(
- (1, Array(1.0, 2.0, 3.0)),
+ (1, Array()),
(2, Array(3.0, 4.0, 5.0)),
(3, Array(4.0, 5.0, 6.0))
)
sc.parallelize(data, numSlices = 2)
.map{ case (k, v) =>
- (new IntWritable(k), new ArrayWritable(classOf[DoubleWritable], v.map(new DoubleWritable(_))))
- }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, ArrayWritable]](arrPath)
+ val va = new DoubleArrayWritable
+ va.set(v.map(new DoubleWritable(_)))
+ (new IntWritable(k), va)
+ }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, DoubleArrayWritable]](arrPath)
// Create test data for MapWritable, with keys DoubleWritable and values Text
val mapData = Seq(
- (1, Map(2.0 -> "aa")),
- (2, Map(3.0 -> "bb")),
+ (1, Map()),
(2, Map(1.0 -> "cc")),
(3, Map(2.0 -> "dd")),
(2, Map(1.0 -> "aa")),
@@ -126,9 +167,9 @@ object WriteInputFormatTestDataGenerator {
)
sc.parallelize(mapData, numSlices = 2).map{ case (i, m) =>
val mw = new MapWritable()
- val k = m.keys.head
- val v = m.values.head
- mw.put(new DoubleWritable(k), new Text(v))
+ m.foreach { case (k, v) =>
+ mw.put(new DoubleWritable(k), new Text(v))
+ }
(new IntWritable(i), mw)
}.saveAsSequenceFile(mapPath)
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index 76956f6a345d1..15fd30e65761d 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -106,23 +106,23 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable {
* Actually get the broadcasted value. Concrete implementations of Broadcast class must
* define their own way to get the value.
*/
- private[spark] def getValue(): T
+ protected def getValue(): T
/**
* Actually unpersist the broadcasted value on the executors. Concrete implementations of
* Broadcast class must define their own logic to unpersist their own data.
*/
- private[spark] def doUnpersist(blocking: Boolean)
+ protected def doUnpersist(blocking: Boolean)
/**
* Actually destroy all data and metadata related to this broadcast variable.
* Implementation of Broadcast class must define their own logic to destroy their own
* state.
*/
- private[spark] def doDestroy(blocking: Boolean)
+ protected def doDestroy(blocking: Boolean)
/** Check if this broadcast is valid. If not valid, exception is thrown. */
- private[spark] def assertValid() {
+ protected def assertValid() {
if (!_isValid) {
throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString))
}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
index c88be6aba6901..8f8a0b11f9f2e 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
@@ -39,7 +39,7 @@ private[spark] class BroadcastManager(
synchronized {
if (!initialized) {
val broadcastFactoryClass =
- conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
+ conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
broadcastFactory =
Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 4f6cabaff2b99..487456467b23b 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -40,9 +40,9 @@ private[spark] class HttpBroadcast[T: ClassTag](
@transient var value_ : T, isLocal: Boolean, id: Long)
extends Broadcast[T](id) with Logging with Serializable {
- def getValue = value_
+ override protected def getValue() = value_
- val blockId = BroadcastBlockId(id)
+ private val blockId = BroadcastBlockId(id)
/*
* Broadcasted data is also stored in the BlockManager of the driver. The BlockManagerMaster
@@ -60,14 +60,14 @@ private[spark] class HttpBroadcast[T: ClassTag](
/**
* Remove all persisted state associated with this HTTP broadcast on the executors.
*/
- def doUnpersist(blocking: Boolean) {
+ override protected def doUnpersist(blocking: Boolean) {
HttpBroadcast.unpersist(id, removeFromDriver = false, blocking)
}
/**
* Remove all persisted state associated with this HTTP broadcast on the executors and driver.
*/
- def doDestroy(blocking: Boolean) {
+ override protected def doDestroy(blocking: Boolean) {
HttpBroadcast.unpersist(id, removeFromDriver = true, blocking)
}
@@ -102,7 +102,7 @@ private[spark] class HttpBroadcast[T: ClassTag](
}
}
-private[spark] object HttpBroadcast extends Logging {
+private[broadcast] object HttpBroadcast extends Logging {
private var initialized = false
private var broadcastDir: File = null
private var compress: Boolean = false
@@ -160,7 +160,7 @@ private[spark] object HttpBroadcast extends Logging {
def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name)
- def write(id: Long, value: Any) {
+ private def write(id: Long, value: Any) {
val file = getFile(id)
val out: OutputStream = {
if (compress) {
@@ -176,7 +176,7 @@ private[spark] object HttpBroadcast extends Logging {
files += file
}
- def read[T: ClassTag](id: Long): T = {
+ private def read[T: ClassTag](id: Long): T = {
logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id)
val url = serverUri + "/" + BroadcastBlockId(id).name
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
index d5a031e2bbb59..c7ef02d572a19 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
@@ -27,21 +27,21 @@ import org.apache.spark.{SecurityManager, SparkConf}
* [[org.apache.spark.broadcast.HttpBroadcast]] for more details about this mechanism.
*/
class HttpBroadcastFactory extends BroadcastFactory {
- def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) {
+ override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) {
HttpBroadcast.initialize(isDriver, conf, securityMgr)
}
- def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) =
+ override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) =
new HttpBroadcast[T](value_, isLocal, id)
- def stop() { HttpBroadcast.stop() }
+ override def stop() { HttpBroadcast.stop() }
/**
* Remove all persisted state associated with the HTTP broadcast with the given ID.
* @param removeFromDriver Whether to remove state from the driver
* @param blocking Whether to block until unbroadcasted
*/
- def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) {
+ override def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) {
HttpBroadcast.unpersist(id, removeFromDriver, blocking)
}
}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 734de37ba115d..86731b684f441 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -20,7 +20,6 @@ package org.apache.spark.broadcast
import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream}
import scala.reflect.ClassTag
-import scala.math
import scala.util.Random
import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException}
@@ -49,19 +48,19 @@ private[spark] class TorrentBroadcast[T: ClassTag](
@transient var value_ : T, isLocal: Boolean, id: Long)
extends Broadcast[T](id) with Logging with Serializable {
- def getValue = value_
+ override protected def getValue() = value_
- val broadcastId = BroadcastBlockId(id)
+ private val broadcastId = BroadcastBlockId(id)
TorrentBroadcast.synchronized {
SparkEnv.get.blockManager.putSingle(
broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false)
}
- @transient var arrayOfBlocks: Array[TorrentBlock] = null
- @transient var totalBlocks = -1
- @transient var totalBytes = -1
- @transient var hasBlocks = 0
+ @transient private var arrayOfBlocks: Array[TorrentBlock] = null
+ @transient private var totalBlocks = -1
+ @transient private var totalBytes = -1
+ @transient private var hasBlocks = 0
if (!isLocal) {
sendBroadcast()
@@ -70,7 +69,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](
/**
* Remove all persisted state associated with this Torrent broadcast on the executors.
*/
- def doUnpersist(blocking: Boolean) {
+ override protected def doUnpersist(blocking: Boolean) {
TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking)
}
@@ -78,11 +77,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](
* Remove all persisted state associated with this Torrent broadcast on the executors
* and driver.
*/
- def doDestroy(blocking: Boolean) {
+ override protected def doDestroy(blocking: Boolean) {
TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking)
}
- def sendBroadcast() {
+ private def sendBroadcast() {
val tInfo = TorrentBroadcast.blockifyObject(value_)
totalBlocks = tInfo.totalBlocks
totalBytes = tInfo.totalBytes
@@ -159,7 +158,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](
hasBlocks = 0
}
- def receiveBroadcast(): Boolean = {
+ private def receiveBroadcast(): Boolean = {
// Receive meta-info about the size of broadcast data,
// the number of chunks it is divided into, etc.
val metaId = BroadcastBlockId(id, "meta")
@@ -211,7 +210,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](
}
-private[spark] object TorrentBroadcast extends Logging {
+private[broadcast] object TorrentBroadcast extends Logging {
private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024
private var initialized = false
private var conf: SparkConf = null
@@ -272,17 +271,19 @@ private[spark] object TorrentBroadcast extends Logging {
* Remove all persisted blocks associated with this torrent broadcast on the executors.
* If removeFromDriver is true, also remove these persisted blocks on the driver.
*/
- def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized {
- SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking)
+ def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = {
+ synchronized {
+ SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking)
+ }
}
}
-private[spark] case class TorrentBlock(
+private[broadcast] case class TorrentBlock(
blockID: Int,
byteArray: Array[Byte])
extends Serializable
-private[spark] case class TorrentInfo(
+private[broadcast] case class TorrentInfo(
@transient arrayOfBlocks: Array[TorrentBlock],
totalBlocks: Int,
totalBytes: Int)
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
index 1de8396a0e17f..ad0f701d7a98f 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
@@ -28,21 +28,21 @@ import org.apache.spark.{SecurityManager, SparkConf}
*/
class TorrentBroadcastFactory extends BroadcastFactory {
- def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) {
+ override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) {
TorrentBroadcast.initialize(isDriver, conf)
}
- def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) =
+ override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) =
new TorrentBroadcast[T](value_, isLocal, id)
- def stop() { TorrentBroadcast.stop() }
+ override def stop() { TorrentBroadcast.stop() }
/**
* Remove all persisted state associated with the torrent broadcast with the given ID.
* @param removeFromDriver Whether to remove state from the driver.
* @param blocking Whether to block until unbroadcasted
*/
- def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) {
+ override def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) {
TorrentBroadcast.unpersist(id, removeFromDriver, blocking)
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
index 86305d2ea8a09..65a1a8fd7e929 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
@@ -22,7 +22,6 @@ private[spark] class ApplicationDescription(
val maxCores: Option[Int],
val memoryPerSlave: Int,
val command: Command,
- val sparkHome: Option[String],
var appUiUrl: String,
val eventLogDir: Option[String] = None)
extends Serializable {
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index c371dc3a51c73..17c507af2652d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -17,8 +17,6 @@
package org.apache.spark.deploy
-import scala.collection.JavaConversions._
-import scala.collection.mutable.Map
import scala.concurrent._
import akka.actor._
@@ -50,9 +48,6 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends
// 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 classPathConf = "spark.driver.extraClassPath"
@@ -65,10 +60,13 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends
cp.split(java.io.File.pathSeparator)
}
- val javaOptionsConf = "spark.driver.extraJavaOptions"
- val javaOpts = sys.props.get(javaOptionsConf)
+ val extraJavaOptsConf = "spark.driver.extraJavaOptions"
+ val extraJavaOpts = sys.props.get(extraJavaOptsConf)
+ .map(Utils.splitCommandString).getOrElse(Seq.empty)
+ val sparkJavaOpts = Utils.sparkJavaOpts(conf)
+ val javaOpts = sparkJavaOpts ++ extraJavaOpts
val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
- driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts)
+ driverArgs.driverOptions, sys.env, classPathEntries, libraryPathEntries, javaOpts)
val driverDescription = new DriverDescription(
driverArgs.jarUrl,
@@ -109,6 +107,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends
// Exception, if present
statusResponse.exception.map { e =>
println(s"Exception from cluster was: $e")
+ e.printStackTrace()
System.exit(-1)
}
System.exit(0)
@@ -141,8 +140,10 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends
*/
object Client {
def main(args: Array[String]) {
- println("WARNING: This client is deprecated and will be removed in a future version of Spark.")
- println("Use ./bin/spark-submit with \"--master spark://host:port\"")
+ if (!sys.props.contains("SPARK_SUBMIT")) {
+ println("WARNING: This client is deprecated and will be removed in a future version of Spark")
+ println("Use ./bin/spark-submit with \"--master spark://host:port\"")
+ }
val conf = new SparkConf()
val driverArgs = new ClientArguments(args)
diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala
index 32f3ba385084f..a2b263544c6a2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Command.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala
@@ -25,5 +25,5 @@ private[spark] case class Command(
environment: Map[String, String],
classPathEntries: Seq[String],
libraryPathEntries: Seq[String],
- extraJavaOptions: Option[String] = None) {
+ javaOpts: Seq[String]) {
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index c4f5e294a393e..696f32a6f5730 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -56,7 +56,6 @@ private[spark] object JsonProtocol {
("cores" -> obj.maxCores) ~
("memoryperslave" -> obj.memoryPerSlave) ~
("user" -> obj.user) ~
- ("sparkhome" -> obj.sparkHome) ~
("command" -> obj.command.toString)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index b050dccb6d57f..318509a67a36f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -27,25 +27,43 @@ import org.apache.spark.executor.ExecutorURLClassLoader
import org.apache.spark.util.Utils
/**
- * Scala code behind the spark-submit script. The script handles setting up the classpath with
- * relevant Spark dependencies and provides a layer over the different cluster managers and deploy
- * modes that Spark supports.
+ * Main gateway of launching a Spark application.
+ *
+ * This program handles setting up the classpath with relevant Spark dependencies and provides
+ * a layer over the different cluster managers and deploy modes that Spark supports.
*/
object SparkSubmit {
+
+ // Cluster managers
private val YARN = 1
private val STANDALONE = 2
private val MESOS = 4
private val LOCAL = 8
private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL
- private var clusterManager: Int = LOCAL
+ // Deploy modes
+ private val CLIENT = 1
+ private val CLUSTER = 2
+ private val ALL_DEPLOY_MODES = CLIENT | CLUSTER
- /**
- * Special primary resource names that represent shells rather than application jars.
- */
+ // A special jar name that indicates the class being run is inside of Spark itself, and therefore
+ // no user jar is needed.
+ private val SPARK_INTERNAL = "spark-internal"
+
+ // Special primary resource names that represent shells rather than application jars.
private val SPARK_SHELL = "spark-shell"
private val PYSPARK_SHELL = "pyspark-shell"
+ // Exposed for testing
+ private[spark] var exitFn: () => Unit = () => System.exit(-1)
+ private[spark] var printStream: PrintStream = System.err
+ private[spark] def printWarning(str: String) = printStream.println("Warning: " + str)
+ private[spark] def printErrorAndExit(str: String) = {
+ printStream.println("Error: " + str)
+ printStream.println("Run with --help for usage help or --verbose for debug output")
+ exitFn()
+ }
+
def main(args: Array[String]) {
val appArgs = new SparkSubmitArguments(args)
if (appArgs.verbose) {
@@ -55,88 +73,78 @@ object SparkSubmit {
launch(childArgs, classpath, sysProps, mainClass, appArgs.verbose)
}
- // Exposed for testing
- private[spark] var printStream: PrintStream = System.err
- private[spark] var exitFn: () => Unit = () => System.exit(-1)
-
- private[spark] def printErrorAndExit(str: String) = {
- printStream.println("Error: " + str)
- printStream.println("Run with --help for usage help or --verbose for debug output")
- exitFn()
- }
- private[spark] def printWarning(str: String) = printStream.println("Warning: " + str)
-
/**
- * @return a tuple containing the arguments for the child, a list of classpath
- * entries for the child, a list of system properties, a list of env vars
- * and the main class for the child
+ * @return a tuple containing
+ * (1) the arguments for the child process,
+ * (2) a list of classpath entries for the child,
+ * (3) a list of system properties and env vars, and
+ * (4) the main class for the child
*/
private[spark] def createLaunchEnv(args: SparkSubmitArguments)
: (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = {
- if (args.master.startsWith("local")) {
- clusterManager = LOCAL
- } else if (args.master.startsWith("yarn")) {
- clusterManager = YARN
- } else if (args.master.startsWith("spark")) {
- clusterManager = STANDALONE
- } else if (args.master.startsWith("mesos")) {
- clusterManager = MESOS
- } else {
- printErrorAndExit("Master must start with yarn, mesos, spark, or local")
- }
- // Because "yarn-cluster" and "yarn-client" encapsulate both the master
- // and deploy mode, we have some logic to infer the master and deploy mode
- // from each other if only one is specified, or exit early if they are at odds.
- if (args.deployMode == null &&
- (args.master == "yarn-standalone" || args.master == "yarn-cluster")) {
- args.deployMode = "cluster"
- }
- if (args.deployMode == "cluster" && args.master == "yarn-client") {
- printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible")
- }
- if (args.deployMode == "client" &&
- (args.master == "yarn-standalone" || args.master == "yarn-cluster")) {
- printErrorAndExit("Deploy mode \"client\" and master \"" + args.master
- + "\" are not compatible")
- }
- if (args.deployMode == "cluster" && args.master.startsWith("yarn")) {
- args.master = "yarn-cluster"
- }
- if (args.deployMode != "cluster" && args.master.startsWith("yarn")) {
- args.master = "yarn-client"
- }
-
- val deployOnCluster = Option(args.deployMode).getOrElse("client") == "cluster"
-
- val childClasspath = new ArrayBuffer[String]()
+ // Values to return
val childArgs = new ArrayBuffer[String]()
+ val childClasspath = new ArrayBuffer[String]()
val sysProps = new HashMap[String, String]()
var childMainClass = ""
- val isPython = args.isPython
- val isYarnCluster = clusterManager == YARN && deployOnCluster
+ // Set the cluster manager
+ val clusterManager: Int = args.master match {
+ case m if m.startsWith("yarn") => YARN
+ case m if m.startsWith("spark") => STANDALONE
+ case m if m.startsWith("mesos") => MESOS
+ case m if m.startsWith("local") => LOCAL
+ case _ => printErrorAndExit("Master must start with yarn, spark, mesos, or local"); -1
+ }
- // For mesos, only client mode is supported
- if (clusterManager == MESOS && deployOnCluster) {
- printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.")
+ // Set the deploy mode; default is client mode
+ var deployMode: Int = args.deployMode match {
+ case "client" | null => CLIENT
+ case "cluster" => CLUSTER
+ case _ => printErrorAndExit("Deploy mode must be either client or cluster"); -1
}
- // For standalone, only client mode is supported
- if (clusterManager == STANDALONE && deployOnCluster) {
- printErrorAndExit("Cluster deploy mode is currently not supported for standalone clusters.")
+ // Because "yarn-cluster" and "yarn-client" encapsulate both the master
+ // and deploy mode, we have some logic to infer the master and deploy mode
+ // from each other if only one is specified, or exit early if they are at odds.
+ if (clusterManager == YARN) {
+ if (args.master == "yarn-standalone") {
+ printWarning("\"yarn-standalone\" is deprecated. Use \"yarn-cluster\" instead.")
+ args.master = "yarn-cluster"
+ }
+ (args.master, args.deployMode) match {
+ case ("yarn-cluster", null) =>
+ deployMode = CLUSTER
+ case ("yarn-cluster", "client") =>
+ printErrorAndExit("Client deploy mode is not compatible with master \"yarn-cluster\"")
+ case ("yarn-client", "cluster") =>
+ printErrorAndExit("Cluster deploy mode is not compatible with master \"yarn-client\"")
+ case (_, mode) =>
+ args.master = "yarn-" + Option(mode).getOrElse("client")
+ }
+
+ // Make sure YARN is included in our build if we're trying to use it
+ if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) {
+ printErrorAndExit(
+ "Could not load YARN classes. " +
+ "This copy of Spark may not have been compiled with YARN support.")
+ }
}
- // For shells, only client mode is applicable
- if (isShell(args.primaryResource) && deployOnCluster) {
- printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.")
+ // The following modes are not supported or applicable
+ (clusterManager, deployMode) match {
+ case (MESOS, CLUSTER) =>
+ printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.")
+ case (_, CLUSTER) if args.isPython =>
+ printErrorAndExit("Cluster deploy mode is currently not supported for python applications.")
+ case (_, CLUSTER) if isShell(args.primaryResource) =>
+ printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.")
+ case _ =>
}
// If we're running a python app, set the main class to our specific python runner
- if (isPython) {
- if (deployOnCluster) {
- printErrorAndExit("Cluster deploy mode is currently not supported for python.")
- }
+ if (args.isPython) {
if (args.primaryResource == PYSPARK_SHELL) {
args.mainClass = "py4j.GatewayServer"
args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0")
@@ -152,120 +160,123 @@ object SparkSubmit {
sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",")
}
- // If we're deploying into YARN, use yarn.Client as a wrapper around the user class
- if (!deployOnCluster) {
- childMainClass = args.mainClass
- if (isUserJar(args.primaryResource)) {
- childClasspath += args.primaryResource
- }
- } else if (clusterManager == YARN) {
- childMainClass = "org.apache.spark.deploy.yarn.Client"
- childArgs += ("--jar", args.primaryResource)
- childArgs += ("--class", args.mainClass)
- }
-
- // Make sure YARN is included in our build if we're trying to use it
- if (clusterManager == YARN) {
- if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) {
- printErrorAndExit("Could not load YARN classes. " +
- "This copy of Spark may not have been compiled with YARN support.")
- }
- }
-
// Special flag to avoid deprecation warnings at the client
sysProps("SPARK_SUBMIT") = "true"
// A list of rules to map each argument to system properties or command-line options in
// each deploy mode; we iterate through these below
val options = List[OptionAssigner](
- OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"),
- OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"),
- OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"),
- OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true,
+
+ // All cluster managers
+ OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"),
+ OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"),
+ OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"),
+
+ // Standalone cluster only
+ OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"),
+ OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"),
+
+ // Yarn client only
+ OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"),
+ OptionAssigner(args.numExecutors, YARN, CLIENT, sysProp = "spark.executor.instances"),
+ OptionAssigner(args.executorCores, YARN, CLIENT, sysProp = "spark.executor.cores"),
+ OptionAssigner(args.files, YARN, CLIENT, sysProp = "spark.yarn.dist.files"),
+ OptionAssigner(args.archives, YARN, CLIENT, sysProp = "spark.yarn.dist.archives"),
+
+ // Yarn cluster only
+ OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"),
+ OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"),
+ OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"),
+ OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"),
+ OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"),
+ OptionAssigner(args.executorCores, YARN, CLUSTER, clOption = "--executor-cores"),
+ OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"),
+ OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"),
+ OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"),
+
+ // Other options
+ OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER,
sysProp = "spark.driver.extraClassPath"),
- OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true,
+ OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER,
sysProp = "spark.driver.extraJavaOptions"),
- OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true,
+ OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER,
sysProp = "spark.driver.extraLibraryPath"),
- OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"),
- OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"),
- OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"),
- OptionAssigner(args.queue, YARN, true, clOption = "--queue"),
- OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"),
- OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"),
- OptionAssigner(args.numExecutors, YARN, false, sysProp = "spark.executor.instances"),
- OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"),
- OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false,
+ OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES,
sysProp = "spark.executor.memory"),
- OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"),
- OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"),
- OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false,
+ OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES,
sysProp = "spark.cores.max"),
- OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"),
- OptionAssigner(args.files, YARN, true, clOption = "--files"),
- OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"),
- OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"),
- OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"),
- OptionAssigner(args.archives, YARN, true, clOption = "--archives"),
- OptionAssigner(args.jars, YARN, true, clOption = "--addJars"),
- OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars")
+ OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES,
+ sysProp = "spark.files")
)
- // For client mode make any added jars immediately visible on the classpath
- if (args.jars != null && !deployOnCluster) {
- for (jar <- args.jars.split(",")) {
- childClasspath += jar
+ // In client mode, launch the application main class directly
+ // In addition, add the main application jar and any added jars (if any) to the classpath
+ if (deployMode == CLIENT) {
+ childMainClass = args.mainClass
+ if (isUserJar(args.primaryResource)) {
+ childClasspath += args.primaryResource
}
+ if (args.jars != null) { childClasspath ++= args.jars.split(",") }
+ if (args.childArgs != null) { childArgs ++= args.childArgs }
}
+
// Map all arguments to command-line options or system properties for our chosen mode
for (opt <- options) {
- if (opt.value != null && deployOnCluster == opt.deployOnCluster &&
+ if (opt.value != null &&
+ (deployMode & opt.deployMode) != 0 &&
(clusterManager & opt.clusterManager) != 0) {
- if (opt.clOption != null) {
- childArgs += (opt.clOption, opt.value)
- }
- if (opt.sysProp != null) {
- sysProps.put(opt.sysProp, opt.value)
- }
+ if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) }
+ if (opt.sysProp != null) { sysProps.put(opt.sysProp, opt.value) }
}
}
// Add the application jar automatically so the user doesn't have to call sc.addJar
// For YARN cluster mode, the jar is already distributed on each node as "app.jar"
// For python files, the primary resource is already distributed as a regular file
- if (!isYarnCluster && !isPython) {
- var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq())
+ val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER
+ if (!isYarnCluster && !args.isPython) {
+ var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty)
if (isUserJar(args.primaryResource)) {
jars = jars ++ Seq(args.primaryResource)
}
sysProps.put("spark.jars", jars.mkString(","))
}
- // Standalone cluster specific configurations
- if (deployOnCluster && clusterManager == STANDALONE) {
+ // In standalone-cluster mode, use Client as a wrapper around the user class
+ if (clusterManager == STANDALONE && deployMode == CLUSTER) {
+ childMainClass = "org.apache.spark.deploy.Client"
if (args.supervise) {
childArgs += "--supervise"
}
- childMainClass = "org.apache.spark.deploy.Client"
childArgs += "launch"
childArgs += (args.master, args.primaryResource, args.mainClass)
+ if (args.childArgs != null) {
+ childArgs ++= args.childArgs
+ }
}
- // Arguments to be passed to user program
- if (args.childArgs != null) {
- if (!deployOnCluster || clusterManager == STANDALONE) {
- childArgs ++= args.childArgs
- } else if (clusterManager == YARN) {
- for (arg <- args.childArgs) {
- childArgs += ("--arg", arg)
- }
+ // In yarn-cluster mode, use yarn.Client as a wrapper around the user class
+ if (clusterManager == YARN && deployMode == CLUSTER) {
+ childMainClass = "org.apache.spark.deploy.yarn.Client"
+ if (args.primaryResource != SPARK_INTERNAL) {
+ childArgs += ("--jar", args.primaryResource)
+ }
+ childArgs += ("--class", args.mainClass)
+ if (args.childArgs != null) {
+ args.childArgs.foreach { arg => childArgs += ("--arg", arg) }
}
}
+ // Properties given with --conf are superceded by other options, but take precedence over
+ // properties in the defaults file.
+ for ((k, v) <- args.sparkProperties) {
+ sysProps.getOrElseUpdate(k, v)
+ }
+
// Read from default spark properties, if any
for ((k, v) <- args.getDefaultSparkProperties) {
- if (!sysProps.contains(k)) sysProps(k) = v
+ sysProps.getOrElseUpdate(k, v)
}
(childArgs, childClasspath, sysProps, childMainClass)
@@ -328,7 +339,7 @@ object SparkSubmit {
* Return whether the given primary resource represents a user jar.
*/
private def isUserJar(primaryResource: String): Boolean = {
- !isShell(primaryResource) && !isPython(primaryResource)
+ !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource)
}
/**
@@ -345,6 +356,10 @@ object SparkSubmit {
primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL
}
+ private[spark] def isInternal(primaryResource: String): Boolean = {
+ primaryResource == SPARK_INTERNAL
+ }
+
/**
* Merge a sequence of comma-separated file lists, some of which may be null to indicate
* no files, into a single comma-separated string.
@@ -364,6 +379,6 @@ object SparkSubmit {
private[spark] case class OptionAssigner(
value: String,
clusterManager: Int,
- deployOnCluster: Boolean,
+ deployMode: Int,
clOption: String = null,
sysProp: String = null)
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index 57655aa4c32b1..9391f24e71ed7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -55,9 +55,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
var verbose: Boolean = false
var isPython: Boolean = false
var pyFiles: String = null
+ val sparkProperties: HashMap[String, String] = new HashMap[String, String]()
parseOpts(args.toList)
- loadDefaults()
+ mergeSparkProperties()
checkRequiredArguments()
/** Return default present in the currently defined defaults file. */
@@ -78,10 +79,23 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
defaultProperties
}
- /** Fill in any undefined values based on the current properties file or built-in defaults. */
- private def loadDefaults(): Unit = {
-
+ /**
+ * Fill in any undefined values based on the default properties file or options passed in through
+ * the '--conf' flag.
+ */
+ private def mergeSparkProperties(): Unit = {
// Use common defaults file, if not specified by user
+ if (propertiesFile == null) {
+ sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir =>
+ val sep = File.separator
+ val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf"
+ val file = new File(defaultPath)
+ if (file.exists()) {
+ propertiesFile = file.getAbsolutePath
+ }
+ }
+ }
+
if (propertiesFile == null) {
sys.env.get("SPARK_HOME").foreach { sparkHome =>
val sep = File.separator
@@ -93,18 +107,20 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
}
}
- val defaultProperties = getDefaultSparkProperties
+ val properties = getDefaultSparkProperties
+ properties.putAll(sparkProperties)
+
// Use properties file as fallback for values which have a direct analog to
// arguments in this script.
- master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull)
+ master = Option(master).getOrElse(properties.get("spark.master").orNull)
executorMemory = Option(executorMemory)
- .getOrElse(defaultProperties.get("spark.executor.memory").orNull)
+ .getOrElse(properties.get("spark.executor.memory").orNull)
executorCores = Option(executorCores)
- .getOrElse(defaultProperties.get("spark.executor.cores").orNull)
+ .getOrElse(properties.get("spark.executor.cores").orNull)
totalExecutorCores = Option(totalExecutorCores)
- .getOrElse(defaultProperties.get("spark.cores.max").orNull)
- name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull)
- jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull)
+ .getOrElse(properties.get("spark.cores.max").orNull)
+ name = Option(name).getOrElse(properties.get("spark.app.name").orNull)
+ jars = Option(jars).getOrElse(properties.get("spark.jars").orNull)
// This supports env vars in older versions of Spark
master = Option(master).getOrElse(System.getenv("MASTER"))
@@ -177,6 +193,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
| executorCores $executorCores
| totalExecutorCores $totalExecutorCores
| propertiesFile $propertiesFile
+ | extraSparkProperties $sparkProperties
| driverMemory $driverMemory
| driverCores $driverCores
| driverExtraClassPath $driverExtraClassPath
@@ -202,8 +219,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
/** Fill in values by parsing user options. */
private def parseOpts(opts: Seq[String]): Unit = {
- // Delineates parsing of Spark options from parsing of user options.
var inSparkOpts = true
+
+ // Delineates parsing of Spark options from parsing of user options.
parse(opts)
def parse(opts: Seq[String]): Unit = opts match {
@@ -290,6 +308,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
jars = Utils.resolveURIs(value)
parse(tail)
+ case ("--conf" | "-c") :: value :: tail =>
+ value.split("=", 2).toSeq match {
+ case Seq(k, v) => sparkProperties(k) = v
+ case _ => SparkSubmit.printErrorAndExit(s"Spark config without '=': $value")
+ }
+ parse(tail)
+
case ("--help" | "-h") :: tail =>
printUsageAndExit(0)
@@ -309,7 +334,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
SparkSubmit.printErrorAndExit(errMessage)
case v =>
primaryResource =
- if (!SparkSubmit.isShell(v)) {
+ if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) {
Utils.resolveURI(v).toString
} else {
v
@@ -349,6 +374,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
| on the PYTHONPATH for Python apps.
| --files FILES Comma-separated list of files to be placed in the working
| directory of each executor.
+ |
+ | --conf PROP=VALUE Arbitrary Spark configuration property.
| --properties-file FILE Path to a file from which to load extra properties. If not
| specified, this will look for conf/spark-defaults.conf.
|
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index e15a87bd38fda..88a0862b96afe 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -46,11 +46,10 @@ private[spark] object TestClient {
def main(args: Array[String]) {
val url = args(0)
val conf = new SparkConf
- val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
+ val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
conf = conf, securityManager = new SecurityManager(conf))
- val desc = new ApplicationDescription(
- "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(),
- Seq()), Some("dummy-spark-home"), "ignored")
+ val desc = new ApplicationDescription("TestClient", Some(1), 512,
+ Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored")
val listener = new TestListener
val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
client.start()
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index a8c9ac072449f..6d2d4cef1ee46 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -36,11 +36,11 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
conf.getInt("spark.history.updateInterval", 10)) * 1000
private val logDir = conf.get("spark.history.fs.logDirectory", null)
- if (logDir == null) {
- throw new IllegalArgumentException("Logging directory must be specified.")
- }
+ private val resolvedLogDir = Option(logDir)
+ .map { d => Utils.resolveURI(d) }
+ .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") }
- private val fs = Utils.getHadoopFileSystem(logDir)
+ private val fs = Utils.getHadoopFileSystem(resolvedLogDir)
// A timestamp of when the disk was last accessed to check for log updates
private var lastLogCheckTimeMs = -1L
@@ -76,14 +76,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
private def initialize() {
// Validate the log directory.
- val path = new Path(logDir)
+ val path = new Path(resolvedLogDir)
if (!fs.exists(path)) {
throw new IllegalArgumentException(
- "Logging directory specified does not exist: %s".format(logDir))
+ "Logging directory specified does not exist: %s".format(resolvedLogDir))
}
if (!fs.getFileStatus(path).isDir) {
throw new IllegalArgumentException(
- "Logging directory specified is not a directory: %s".format(logDir))
+ "Logging directory specified is not a directory: %s".format(resolvedLogDir))
}
checkForLogs()
@@ -95,15 +95,16 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
override def getAppUI(appId: String): SparkUI = {
try {
- val appLogDir = fs.getFileStatus(new Path(logDir, appId))
- loadAppInfo(appLogDir, true)._2
+ val appLogDir = fs.getFileStatus(new Path(resolvedLogDir.toString, appId))
+ val (_, ui) = loadAppInfo(appLogDir, renderUI = true)
+ ui
} catch {
case e: FileNotFoundException => null
}
}
override def getConfig(): Map[String, String] =
- Map(("Event Log Location" -> logDir))
+ Map("Event Log Location" -> resolvedLogDir.toString)
/**
* Builds the application list based on the current contents of the log directory.
@@ -114,14 +115,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
lastLogCheckTimeMs = getMonotonicTimeMs()
logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
try {
- val logStatus = fs.listStatus(new Path(logDir))
+ val logStatus = fs.listStatus(new Path(resolvedLogDir))
val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
- val logInfos = logDirs.filter {
- dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
+ val logInfos = logDirs.filter { dir =>
+ fs.isFile(new Path(dir.getPath, EventLoggingListener.APPLICATION_COMPLETE))
}
val currentApps = Map[String, ApplicationHistoryInfo](
- appList.map(app => (app.id -> app)):_*)
+ appList.map(app => app.id -> app):_*)
// For any application that either (i) is not listed or (ii) has changed since the last time
// the listing was created (defined by the log dir's modification time), load the app's info.
@@ -131,7 +132,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
val curr = currentApps.getOrElse(dir.getPath().getName(), null)
if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
try {
- newApps += loadAppInfo(dir, false)._1
+ val (app, _) = loadAppInfo(dir, renderUI = false)
+ newApps += app
} catch {
case e: Exception => logError(s"Failed to load app info from directory $dir.")
}
@@ -159,9 +161,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
* @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
*/
private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = {
- val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
val path = logDir.getPath
val appId = path.getName
+ val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs)
val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
val appListener = new ApplicationEventListener
replayBus.addListener(appListener)
@@ -169,7 +171,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
val ui: SparkUI = if (renderUI) {
val conf = this.conf.clone()
val appSecManager = new SecurityManager(conf)
- new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
+ new SparkUI(conf, appSecManager, replayBus, appId,
+ HistoryServer.UI_PATH_PREFIX + s"/$appId")
// Do not call ui.bind() to avoid creating a new server for each application
} else {
null
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
index a958c837c2ff6..c4ef8b63b0071 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
@@ -45,7 +45,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
- { providerConfig.map(e => - {e._1}: {e._2}
) }
+ {providerConfig.map { case (k, v) => - {k}: {v}
}}
{
if (allApps.size > 0) {
@@ -75,7 +75,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
"Last Updated")
private def appRow(info: ApplicationHistoryInfo): Seq[Node] = {
- val uiAddress = "/history/" + info.id
+ val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}"
val startTime = UIUtils.formatDate(info.startTime)
val endTime = UIUtils.formatDate(info.endTime)
val duration = UIUtils.formatDuration(info.endTime - info.startTime)
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 56b38ddfc9313..d1a64c1912cb8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -25,9 +25,9 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.ui.{WebUI, SparkUI, UIUtils}
+import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.util.{SignalLogger, Utils}
+import org.apache.spark.util.SignalLogger
/**
* A web server that renders SparkUIs of completed applications.
@@ -114,7 +114,7 @@ class HistoryServer(
attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
val contextHandler = new ServletContextHandler
- contextHandler.setContextPath("/history")
+ contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX)
contextHandler.addServlet(new ServletHolder(loaderServlet), "/*")
attachHandler(contextHandler)
}
@@ -172,10 +172,12 @@ class HistoryServer(
object HistoryServer extends Logging {
private val conf = new SparkConf
+ val UI_PATH_PREFIX = "/history"
+
def main(argStrings: Array[String]) {
SignalLogger.register(log)
initSecurity()
- val args = new HistoryServerArguments(conf, argStrings)
+ new HistoryServerArguments(conf, argStrings)
val securityManager = new SecurityManager(conf)
val providerName = conf.getOption("spark.history.provider")
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
index be9361b754fc3..25fc76c23e0fb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
@@ -18,7 +18,6 @@
package org.apache.spark.deploy.history
import org.apache.spark.SparkConf
-import org.apache.spark.util.Utils
/**
* Command-line parser for the master.
@@ -32,6 +31,7 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]
args match {
case ("--dir" | "-d") :: value :: tail =>
logDir = value
+ conf.set("spark.history.fs.logDirectory", value)
parse(tail)
case ("--help" | "-h") :: tail =>
@@ -42,9 +42,6 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]
case _ =>
printUsageAndExit(1)
}
- if (logDir != null) {
- conf.set("spark.history.fs.logDirectory", logDir)
- }
}
private def printUsageAndExit(exitCode: Int) {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index a304102a49086..a70ecdb375373 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -17,6 +17,7 @@
package org.apache.spark.deploy.master
+import java.net.URLEncoder
import java.text.SimpleDateFormat
import java.util.Date
@@ -30,11 +31,11 @@ import akka.actor._
import akka.pattern.ask
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import akka.serialization.SerializationExtension
-import org.apache.hadoop.fs.FileSystem
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.history.HistoryServer
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.deploy.master.MasterMessages._
import org.apache.spark.deploy.master.ui.MasterWebUI
@@ -57,6 +58,7 @@ private[spark] class Master(
def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000
val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200)
+ val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 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")
@@ -72,9 +74,7 @@ private[spark] class Master(
val waitingApps = new ArrayBuffer[ApplicationInfo]
val completedApps = new ArrayBuffer[ApplicationInfo]
var nextAppNumber = 0
-
val appIdToUI = new HashMap[String, SparkUI]
- val fileSystemsUsed = new HashSet[FileSystem]
val drivers = new HashSet[DriverInfo]
val completedDrivers = new ArrayBuffer[DriverInfo]
@@ -154,12 +154,13 @@ private[spark] class Master(
}
override def postStop() {
+ masterMetricsSystem.report()
+ applicationMetricsSystem.report()
// prevent the CompleteRecovery message sending to restarted master
if (recoveryCompletionTask != null) {
recoveryCompletionTask.cancel()
}
webUi.stop()
- fileSystemsUsed.foreach(_.close())
masterMetricsSystem.stop()
applicationMetricsSystem.stop()
persistenceEngine.close()
@@ -644,10 +645,7 @@ private[spark] class Master(
waitingApps -= app
// If application events are logged, use them to rebuild the UI
- if (!rebuildSparkUI(app)) {
- // Avoid broken links if the UI is not reconstructed
- app.desc.appUiUrl = ""
- }
+ rebuildSparkUI(app)
for (exec <- app.executors.values) {
exec.worker.removeExecutor(exec)
@@ -669,29 +667,49 @@ private[spark] class Master(
*/
def rebuildSparkUI(app: ApplicationInfo): Boolean = {
val appName = app.desc.name
- val eventLogDir = app.desc.eventLogDir.getOrElse { return false }
+ val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
+ val eventLogDir = app.desc.eventLogDir.getOrElse {
+ // Event logging is not enabled for this application
+ app.desc.appUiUrl = notFoundBasePath
+ return false
+ }
val fileSystem = Utils.getHadoopFileSystem(eventLogDir)
val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem)
val eventLogPaths = eventLogInfo.logPaths
val compressionCodec = eventLogInfo.compressionCodec
- if (!eventLogPaths.isEmpty) {
- try {
- val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
- val ui = new SparkUI(
- new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id)
- replayBus.replay()
- app.desc.appUiUrl = ui.basePath
- appIdToUI(app.id) = ui
- webUi.attachSparkUI(ui)
- return true
- } catch {
- case e: Exception =>
- logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e)
- }
- } else {
- logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir))
+
+ if (eventLogPaths.isEmpty) {
+ // Event logging is enabled for this application, but no event logs are found
+ val title = s"Application history not found (${app.id})"
+ var msg = s"No event logs found for application $appName in $eventLogDir."
+ logWarning(msg)
+ msg += " Did you specify the correct logging directory?"
+ msg = URLEncoder.encode(msg, "UTF-8")
+ app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
+ return false
+ }
+
+ try {
+ val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
+ val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)",
+ HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
+ replayBus.replay()
+ appIdToUI(app.id) = ui
+ webUi.attachSparkUI(ui)
+ // Application UI is successfully rebuilt, so link the Master UI to it
+ app.desc.appUiUrl = ui.basePath
+ true
+ } catch {
+ case e: Exception =>
+ // Relay exception message to application UI page
+ val title = s"Application history load error (${app.id})"
+ val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8")
+ var msg = s"Exception in replaying log for application $appName!"
+ logError(msg, e)
+ msg = URLEncoder.encode(msg, "UTF-8")
+ app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title"
+ false
}
- false
}
/** Generate a new app ID given a app's submission date */
@@ -744,11 +762,16 @@ private[spark] class Master(
case Some(driver) =>
logInfo(s"Removing driver: $driverId")
drivers -= driver
+ if (completedDrivers.size >= RETAINED_DRIVERS) {
+ val toRemove = math.max(RETAINED_DRIVERS / 10, 1)
+ completedDrivers.trimStart(toRemove)
+ }
completedDrivers += driver
persistenceEngine.removeDriver(driver)
driver.state = finalState
driver.exception = exception
driver.worker.foreach(w => w.removeDriver(driver))
+ schedule()
case None =>
logWarning(s"Asked to remove unknown driver: $driverId")
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 34fa1429c86de..4588c130ef439 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -28,7 +28,7 @@ import org.json4s.JValue
import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import org.apache.spark.deploy.master.ExecutorInfo
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
import org.apache.spark.util.Utils
private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala
new file mode 100644
index 0000000000000..d8daff3e7fb9c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.ui
+
+import java.net.URLDecoder
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+
+private[spark] class HistoryNotFoundPage(parent: MasterWebUI)
+ extends WebUIPage("history/not-found") {
+
+ /**
+ * Render a page that conveys failure in loading application history.
+ *
+ * This accepts 3 HTTP parameters:
+ * msg = message to display to the user
+ * title = title of the page
+ * exception = detailed description of the exception in loading application history (if any)
+ *
+ * Parameters "msg" and "exception" are assumed to be UTF-8 encoded.
+ */
+ def render(request: HttpServletRequest): Seq[Node] = {
+ val titleParam = request.getParameter("title")
+ val msgParam = request.getParameter("msg")
+ val exceptionParam = request.getParameter("exception")
+
+ // If no parameters are specified, assume the user did not enable event logging
+ val defaultTitle = "Event logging is not enabled"
+ val defaultContent =
+
+
+ No event logs were found for this application! To
+
enable event logging,
+ set
spark.eventLog.enabled to true and
+
spark.eventLog.dir to the directory to which your
+ event logs are written.
+
+
+
+ val title = Option(titleParam).getOrElse(defaultTitle)
+ val content = Option(msgParam)
+ .map { msg => URLDecoder.decode(msg, "UTF-8") }
+ .map { msg =>
+
++
+ Option(exceptionParam)
+ .map { e => URLDecoder.decode(e, "UTF-8") }
+ .map { e =>
{e}
}
+ .getOrElse(Seq.empty)
+ }.getOrElse(defaultContent)
+
+ UIUtils.basicSparkPage(content, title)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index a18b39fc95d64..16aa0493370dd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -21,7 +21,7 @@ import org.apache.spark.Logging
import org.apache.spark.deploy.master.Master
import org.apache.spark.ui.{SparkUI, WebUI}
import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.util.AkkaUtils
/**
* Web UI server for the standalone master.
@@ -38,6 +38,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
/** Initialize all components of the server. */
def initialize() {
attachPage(new ApplicationPage(this))
+ attachPage(new HistoryNotFoundPage(this))
attachPage(new MasterPage(this))
attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
master.masterMetricsSystem.getServletHandlers.foreach(attachHandler)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index 4af5bc3afad6c..687e492a0d6fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -47,7 +47,6 @@ object CommandUtils extends Logging {
*/
def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = {
val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M")
- val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq())
// Exists for backwards compatibility with older Spark versions
val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString)
@@ -62,7 +61,7 @@ object CommandUtils extends Logging {
val joined = command.libraryPathEntries.mkString(File.pathSeparator)
Seq(s"-Djava.library.path=$joined")
} else {
- Seq()
+ Seq()
}
val permGenOpt = Seq("-XX:MaxPermSize=128m")
@@ -71,11 +70,11 @@ object CommandUtils extends Logging {
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)
+ extraEnvironment = command.environment)
val userClassPath = command.classPathEntries ++ Seq(classPath)
Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++
- permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts
+ permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts
}
/** Spawn a thread that will redirect a given stream to a file */
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 662d37871e7a6..5caaf6bea3575 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -36,6 +36,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState
/**
* Manages the execution of one driver, including automatically restarting the driver on failure.
+ * This is currently only used in standalone cluster deploy mode.
*/
private[spark] class DriverRunner(
val driverId: String,
@@ -81,7 +82,7 @@ private[spark] class DriverRunner(
driverDesc.command.environment,
classPath,
driverDesc.command.libraryPathEntries,
- driverDesc.command.extraJavaOptions)
+ driverDesc.command.javaOpts)
val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
sparkHome.getAbsolutePath)
launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index 467317dd9b44c..7be89f9aff0f3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -30,6 +30,7 @@ import org.apache.spark.util.logging.FileAppender
/**
* Manages the execution of one executor process.
+ * This is currently only used in standalone mode.
*/
private[spark] class ExecutorRunner(
val appId: String,
@@ -72,7 +73,7 @@ private[spark] class ExecutorRunner(
}
/**
- * kill executor process, wait for exit and notify worker to update resource status
+ * Kill executor process, wait for exit and notify worker to update resource status.
*
* @param message the exception message which caused the executor's death
*/
@@ -114,10 +115,13 @@ private[spark] class ExecutorRunner(
}
def getCommandSeq = {
- val command = Command(appDesc.command.mainClass,
- appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment,
- appDesc.command.classPathEntries, appDesc.command.libraryPathEntries,
- appDesc.command.extraJavaOptions)
+ val command = Command(
+ appDesc.command.mainClass,
+ appDesc.command.arguments.map(substituteVariables) ++ Seq(appId),
+ appDesc.command.environment,
+ appDesc.command.classPathEntries,
+ appDesc.command.libraryPathEntries,
+ appDesc.command.javaOpts)
CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index ce425443051b0..458d9947bd873 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -71,7 +71,7 @@ private[spark] class Worker(
// TTL for app folders/data; after TTL expires it will be cleaned up
val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600)
-
+ val testing: Boolean = sys.props.contains("spark.testing")
val masterLock: Object = new Object()
var master: ActorSelection = null
var masterAddress: Address = null
@@ -81,7 +81,13 @@ private[spark] class Worker(
@volatile var registered = false
@volatile var connected = false
val workerId = generateWorkerId()
- val sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
+ val sparkHome =
+ if (testing) {
+ assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!")
+ new File(sys.props("spark.test.home"))
+ } else {
+ new File(sys.env.get("SPARK_HOME").getOrElse("."))
+ }
var workDir: File = null
val executors = new HashMap[String, ExecutorRunner]
val finishedExecutors = new HashMap[String, ExecutorRunner]
@@ -233,9 +239,7 @@ private[spark] class Worker(
try {
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,
- appDesc.sparkHome.map(userSparkHome => new File(userSparkHome)).getOrElse(sparkHome),
- workDir, akkaUrl, conf, ExecutorState.RUNNING)
+ self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
@@ -357,6 +361,7 @@ private[spark] class Worker(
}
override def postStop() {
+ metricsSystem.report()
registrationRetryTimer.foreach(_.cancel())
executors.values.foreach(_.kill())
drivers.values.foreach(_.kill())
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
index b389cb546de6c..ecb358c399819 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
@@ -17,7 +17,6 @@
package org.apache.spark.deploy.worker.ui
-import java.io.File
import javax.servlet.http.HttpServletRequest
import scala.xml.Node
@@ -25,7 +24,7 @@ import scala.xml.Node
import org.apache.spark.ui.{WebUIPage, UIUtils}
import org.apache.spark.util.Utils
import org.apache.spark.Logging
-import org.apache.spark.util.logging.{FileAppender, RollingFileAppender}
+import org.apache.spark.util.logging.RollingFileAppender
private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging {
private val worker = parent.worker
@@ -64,11 +63,11 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w
val offset = Option(request.getParameter("offset")).map(_.toLong)
val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes)
- val (logDir, params) = (appId, executorId, driverId) match {
+ val (logDir, params, pageName) = (appId, executorId, driverId) match {
case (Some(a), Some(e), None) =>
- (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e")
+ (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e", s"$a/$e")
case (None, None, Some(d)) =>
- (s"${workDir.getPath}/$d/", s"driverId=$d")
+ (s"${workDir.getPath}/$d/", s"driverId=$d", d)
case _ =>
throw new Exception("Request must specify either application or driver identifiers")
}
@@ -120,7 +119,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w