diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 9b62e4b1b715..48d3630abd1f 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -213,13 +213,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
driverConf.set(key, value)
}
}
- if (driverConf.contains("spark.yarn.credentials.file")) {
- logInfo("Will periodically update credentials from: " +
- driverConf.get("spark.yarn.credentials.file"))
- Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
- .getMethod("startCredentialUpdater", classOf[SparkConf])
- .invoke(null, driverConf)
- }
cfg.hadoopDelegationCreds.foreach { tokens =>
SparkHadoopUtil.get.addDelegationTokens(tokens, driverConf)
@@ -234,11 +227,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url))
}
env.rpcEnv.awaitTermination()
- if (driverConf.contains("spark.yarn.credentials.file")) {
- Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
- .getMethod("stopCredentialUpdater")
- .invoke(null)
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index 2c3a8ef74800..c325222b764b 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -35,6 +35,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
@@ -141,8 +142,7 @@ private[spark] class Executor(
conf.getSizeAsBytes("spark.task.maxDirectResultSize", 1L << 20),
RpcUtils.maxMessageSizeBytes(conf))
- // Limit of bytes for total size of results (default is 1GB)
- private val maxResultSize = Utils.getMaxResultSize(conf)
+ private val maxResultSize = conf.get(MAX_RESULT_SIZE)
// Maintains the list of running tasks.
private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
@@ -480,6 +480,19 @@ private[spark] class Executor(
execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
} catch {
+ case t: TaskKilledException =>
+ logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}")
+ setTaskFinishedAndClearInterruptStatus()
+ execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason)))
+
+ case _: InterruptedException | NonFatal(_) if
+ task != null && task.reasonIfKilled.isDefined =>
+ val killReason = task.reasonIfKilled.getOrElse("unknown reason")
+ logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason")
+ setTaskFinishedAndClearInterruptStatus()
+ execBackend.statusUpdate(
+ taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason)))
+
case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) =>
val reason = task.context.fetchFailed.get.toTaskFailedReason
if (!t.isInstanceOf[FetchFailedException]) {
@@ -494,19 +507,6 @@ private[spark] class Executor(
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
- case t: TaskKilledException =>
- logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}")
- setTaskFinishedAndClearInterruptStatus()
- execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason)))
-
- case _: InterruptedException | NonFatal(_) if
- task != null && task.reasonIfKilled.isDefined =>
- val killReason = task.reasonIfKilled.getOrElse("unknown reason")
- logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason")
- setTaskFinishedAndClearInterruptStatus()
- execBackend.statusUpdate(
- taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason)))
-
case CausedBy(cDE: CommitDeniedException) =>
val reason = cDE.toTaskCommitDeniedReason
setTaskFinishedAndClearInterruptStatus()
diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
index b0cd7110a3b4..f27aca03773a 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
@@ -23,6 +23,7 @@ import java.util.regex.PatternSyntaxException
import scala.util.matching.Regex
import org.apache.spark.network.util.{ByteUnit, JavaUtils}
+import org.apache.spark.util.Utils
private object ConfigHelpers {
@@ -45,7 +46,7 @@ private object ConfigHelpers {
}
def stringToSeq[T](str: String, converter: String => T): Seq[T] = {
- str.split(",").map(_.trim()).filter(_.nonEmpty).map(converter)
+ Utils.stringToSeq(str).map(converter)
}
def seqToString[T](v: Seq[T], stringConverter: T => String): String = {
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index eb12ddf96131..6bb98c37b447 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -38,10 +38,13 @@ package object config {
ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false)
private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory")
+ .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.")
.bytesConf(ByteUnit.MiB)
.createWithDefaultString("1g")
private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead")
+ .doc("The amount of off-heap memory to be allocated per driver in cluster mode, " +
+ "in MiB unless otherwise specified.")
.bytesConf(ByteUnit.MiB)
.createOptional
@@ -62,6 +65,7 @@ package object config {
.createWithDefault(false)
private[spark] val EVENT_LOG_OUTPUT_BUFFER_SIZE = ConfigBuilder("spark.eventLog.buffer.kb")
+ .doc("Buffer size to use when writing to output streams, in KiB unless otherwise specified.")
.bytesConf(ByteUnit.KiB)
.createWithDefaultString("100k")
@@ -81,10 +85,13 @@ package object config {
ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false)
private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory")
+ .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.")
.bytesConf(ByteUnit.MiB)
.createWithDefaultString("1g")
private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead")
+ .doc("The amount of off-heap memory to be allocated per executor in cluster mode, " +
+ "in MiB unless otherwise specified.")
.bytesConf(ByteUnit.MiB)
.createOptional
@@ -119,6 +126,10 @@ package object config {
private[spark] val DYN_ALLOCATION_MAX_EXECUTORS =
ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.createWithDefault(Int.MaxValue)
+ private[spark] val DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO =
+ ConfigBuilder("spark.dynamicAllocation.executorAllocationRatio")
+ .doubleConf.createWithDefault(1.0)
+
private[spark] val LOCALITY_WAIT = ConfigBuilder("spark.locality.wait")
.timeConf(TimeUnit.MILLISECONDS)
.createWithDefaultString("3s")
@@ -294,6 +305,12 @@ package object config {
.booleanConf
.createWithDefault(false)
+ private[spark] val IGNORE_MISSING_FILES = ConfigBuilder("spark.files.ignoreMissingFiles")
+ .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " +
+ "encountering missing files and the contents that have been read will still be returned.")
+ .booleanConf
+ .createWithDefault(false)
+
private[spark] val APP_CALLER_CONTEXT = ConfigBuilder("spark.log.callerContext")
.stringConf
.createOptional
@@ -353,7 +370,7 @@ package object config {
private[spark] val BUFFER_WRITE_CHUNK_SIZE =
ConfigBuilder("spark.buffer.write.chunkSize")
.internal()
- .doc("The chunk size during writing out the bytes of ChunkedByteBuffer.")
+ .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.")
.bytesConf(ByteUnit.BYTE)
.checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" +
" ChunkedByteBuffer should not larger than Int.MaxValue.")
@@ -368,9 +385,9 @@ package object config {
private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
ConfigBuilder("spark.shuffle.accurateBlockThreshold")
- .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
- "record the size accurately if it's above this config. This helps to prevent OOM by " +
- "avoiding underestimating shuffle block size when fetch shuffle blocks.")
+ .doc("Threshold in bytes above which the size of shuffle blocks in " +
+ "HighlyCompressedMapStatus is accurately recorded. This helps to prevent OOM " +
+ "by avoiding underestimating shuffle block size when fetch shuffle blocks.")
.bytesConf(ByteUnit.BYTE)
.createWithDefault(100 * 1024 * 1024)
@@ -389,23 +406,23 @@ package object config {
private[spark] val REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS =
ConfigBuilder("spark.reducer.maxBlocksInFlightPerAddress")
- .doc("This configuration limits the number of remote blocks being fetched per reduce task" +
- " from a given host port. When a large number of blocks are being requested from a given" +
- " address in a single fetch or simultaneously, this could crash the serving executor or" +
- " Node Manager. This is especially useful to reduce the load on the Node Manager when" +
- " external shuffle is enabled. You can mitigate the issue by setting it to a lower value.")
+ .doc("This configuration limits the number of remote blocks being fetched per reduce task " +
+ "from a given host port. When a large number of blocks are being requested from a given " +
+ "address in a single fetch or simultaneously, this could crash the serving executor or " +
+ "Node Manager. This is especially useful to reduce the load on the Node Manager when " +
+ "external shuffle is enabled. You can mitigate the issue by setting it to a lower value.")
.intConf
.checkValue(_ > 0, "The max no. of blocks in flight cannot be non-positive.")
.createWithDefault(Int.MaxValue)
private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM =
ConfigBuilder("spark.maxRemoteBlockSizeFetchToMem")
- .doc("Remote block will be fetched to disk when size of the block is " +
- "above this threshold. This is to avoid a giant request takes too much memory. We can " +
- "enable this config by setting a specific value(e.g. 200m). Note this configuration will " +
- "affect both shuffle fetch and block manager remote block fetch. For users who " +
- "enabled external shuffle service, this feature can only be worked when external shuffle" +
- " service is newer than Spark 2.2.")
+ .doc("Remote block will be fetched to disk when size of the block is above this threshold " +
+ "in bytes. This is to avoid a giant request takes too much memory. We can enable this " +
+ "config by setting a specific value(e.g. 200m). Note this configuration will affect " +
+ "both shuffle fetch and block manager remote block fetch. For users who enabled " +
+ "external shuffle service, this feature can only be worked when external shuffle" +
+ "service is newer than Spark 2.2.")
.bytesConf(ByteUnit.BYTE)
.createWithDefault(Long.MaxValue)
@@ -419,9 +436,9 @@ package object config {
private[spark] val SHUFFLE_FILE_BUFFER_SIZE =
ConfigBuilder("spark.shuffle.file.buffer")
- .doc("Size of the in-memory buffer for each shuffle file output stream. " +
- "These buffers reduce the number of disk seeks and system calls made " +
- "in creating intermediate shuffle files.")
+ .doc("Size of the in-memory buffer for each shuffle file output stream, in KiB unless " +
+ "otherwise specified. These buffers reduce the number of disk seeks and system calls " +
+ "made in creating intermediate shuffle files.")
.bytesConf(ByteUnit.KiB)
.checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.")
@@ -430,7 +447,7 @@ package object config {
private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE =
ConfigBuilder("spark.shuffle.unsafe.file.output.buffer")
.doc("The file system for this buffer size after each partition " +
- "is written in unsafe shuffle writer.")
+ "is written in unsafe shuffle writer. In KiB unless otherwise specified.")
.bytesConf(ByteUnit.KiB)
.checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.")
@@ -438,7 +455,7 @@ package object config {
private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE =
ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize")
- .doc("The buffer size to use when writing the sorted records to an on-disk file.")
+ .doc("The buffer size, in bytes, to use when writing the sorted records to an on-disk file.")
.bytesConf(ByteUnit.BYTE)
.checkValue(v => v > 0 && v <= Int.MaxValue,
s"The buffer size must be greater than 0 and less than ${Int.MaxValue}.")
@@ -513,4 +530,21 @@ package object config {
.checkValue(v => v > 0, "The threshold should be positive.")
.createWithDefault(10000000)
+ private[spark] val MAX_RESULT_SIZE = ConfigBuilder("spark.driver.maxResultSize")
+ .doc("Size limit for results.")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("1g")
+
+ private[spark] val CREDENTIALS_RENEWAL_INTERVAL_RATIO =
+ ConfigBuilder("spark.security.credentials.renewalRatio")
+ .doc("Ratio of the credential's expiration time when Spark should fetch new credentials.")
+ .doubleConf
+ .createWithDefault(0.75d)
+
+ private[spark] val CREDENTIALS_RENEWAL_RETRY_WAIT =
+ ConfigBuilder("spark.security.credentials.retryWait")
+ .doc("How long to wait before retrying to fetch new credentials after a failure.")
+ .timeConf(TimeUnit.SECONDS)
+ .createWithDefaultString("1h")
+
}
diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
index 50f51e1af453..e6e9c9e32885 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
@@ -20,6 +20,7 @@ package org.apache.spark.internal.io
import org.apache.hadoop.fs._
import org.apache.hadoop.mapreduce._
+import org.apache.spark.internal.Logging
import org.apache.spark.util.Utils
@@ -28,8 +29,9 @@ import org.apache.spark.util.Utils
*
* 1. Implementations must be serializable, as the committer instance instantiated on the driver
* will be used for tasks on executors.
- * 2. Implementations should have a constructor with 2 arguments:
- * (jobId: String, path: String)
+ * 2. Implementations should have a constructor with 2 or 3 arguments:
+ * (jobId: String, path: String) or
+ * (jobId: String, path: String, dynamicPartitionOverwrite: Boolean)
* 3. A committer should not be reused across multiple Spark jobs.
*
* The proper call sequence is:
@@ -131,7 +133,7 @@ abstract class FileCommitProtocol {
}
-object FileCommitProtocol {
+object FileCommitProtocol extends Logging {
class TaskCommitMessage(val obj: Any) extends Serializable
object EmptyTaskCommitMessage extends TaskCommitMessage(null)
@@ -139,10 +141,30 @@ object FileCommitProtocol {
/**
* Instantiates a FileCommitProtocol using the given className.
*/
- def instantiate(className: String, jobId: String, outputPath: String)
- : FileCommitProtocol = {
+ def instantiate(
+ className: String,
+ jobId: String,
+ outputPath: String,
+ dynamicPartitionOverwrite: Boolean = false): FileCommitProtocol = {
+
+ logDebug(s"Creating committer $className; job $jobId; output=$outputPath;" +
+ s" dynamic=$dynamicPartitionOverwrite")
val clazz = Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]]
- val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String])
- ctor.newInstance(jobId, outputPath)
+ // First try the constructor with arguments (jobId: String, outputPath: String,
+ // dynamicPartitionOverwrite: Boolean).
+ // If that doesn't exist, try the one with (jobId: string, outputPath: String).
+ try {
+ val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String], classOf[Boolean])
+ logDebug("Using (String, String, Boolean) constructor")
+ ctor.newInstance(jobId, outputPath, dynamicPartitionOverwrite.asInstanceOf[java.lang.Boolean])
+ } catch {
+ case _: NoSuchMethodException =>
+ logDebug("Falling back to (String, String) constructor")
+ require(!dynamicPartitionOverwrite,
+ "Dynamic Partition Overwrite is enabled but" +
+ s" the committer ${className} does not have the appropriate constructor")
+ val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String])
+ ctor.newInstance(jobId, outputPath)
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index 95c99d29c3a9..3e60c50ada59 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ -39,8 +39,19 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil
*
* @param jobId the job's or stage's id
* @param path the job's output path, or null if committer acts as a noop
+ * @param dynamicPartitionOverwrite If true, Spark will overwrite partition directories at runtime
+ * dynamically, i.e., we first write files under a staging
+ * directory with partition path, e.g.
+ * /path/to/staging/a=1/b=1/xxx.parquet. When committing the job,
+ * we first clean up the corresponding partition directories at
+ * destination path, e.g. /path/to/destination/a=1/b=1, and move
+ * files from staging directory to the corresponding partition
+ * directories under destination path.
*/
-class HadoopMapReduceCommitProtocol(jobId: String, path: String)
+class HadoopMapReduceCommitProtocol(
+ jobId: String,
+ path: String,
+ dynamicPartitionOverwrite: Boolean = false)
extends FileCommitProtocol with Serializable with Logging {
import FileCommitProtocol._
@@ -67,9 +78,17 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
@transient private var addedAbsPathFiles: mutable.Map[String, String] = null
/**
- * The staging directory for all files committed with absolute output paths.
+ * Tracks partitions with default path that have new files written into them by this task,
+ * e.g. a=1/b=2. Files under these partitions will be saved into staging directory and moved to
+ * destination directory at the end, if `dynamicPartitionOverwrite` is true.
*/
- private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId)
+ @transient private var partitionPaths: mutable.Set[String] = null
+
+ /**
+ * The staging directory of this write job. Spark uses it to deal with files with absolute output
+ * path, or writing data into partitioned directory with dynamicPartitionOverwrite=true.
+ */
+ private def stagingDir = new Path(path, ".spark-staging-" + jobId)
protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = {
val format = context.getOutputFormatClass.newInstance()
@@ -85,11 +104,16 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = {
val filename = getFilename(taskContext, ext)
- val stagingDir: String = committer match {
+ val stagingDir: Path = committer match {
+ case _ if dynamicPartitionOverwrite =>
+ assert(dir.isDefined,
+ "The dataset to be written must be partitioned when dynamicPartitionOverwrite is true.")
+ partitionPaths += dir.get
+ this.stagingDir
// For FileOutputCommitter it has its own staging path called "work path".
case f: FileOutputCommitter =>
- Option(f.getWorkPath).map(_.toString).getOrElse(path)
- case _ => path
+ new Path(Option(f.getWorkPath).map(_.toString).getOrElse(path))
+ case _ => new Path(path)
}
dir.map { d =>
@@ -106,8 +130,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
// Include a UUID here to prevent file collisions for one task writing to different dirs.
// In principle we could include hash(absoluteDir) instead but this is simpler.
- val tmpOutputPath = new Path(
- absPathStagingDir, UUID.randomUUID().toString() + "-" + filename).toString
+ val tmpOutputPath = new Path(stagingDir, UUID.randomUUID().toString() + "-" + filename).toString
addedAbsPathFiles(tmpOutputPath) = absOutputPath
tmpOutputPath
@@ -141,23 +164,52 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = {
committer.commitJob(jobContext)
- val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]])
- .foldLeft(Map[String, String]())(_ ++ _)
- logDebug(s"Committing files staged for absolute locations $filesToMove")
+
if (hasValidPath) {
- val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
+ val (allAbsPathFiles, allPartitionPaths) =
+ taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip
+ val fs = stagingDir.getFileSystem(jobContext.getConfiguration)
+
+ val filesToMove = allAbsPathFiles.foldLeft(Map[String, String]())(_ ++ _)
+ logDebug(s"Committing files staged for absolute locations $filesToMove")
+ if (dynamicPartitionOverwrite) {
+ val absPartitionPaths = filesToMove.values.map(new Path(_).getParent).toSet
+ logDebug(s"Clean up absolute partition directories for overwriting: $absPartitionPaths")
+ absPartitionPaths.foreach(fs.delete(_, true))
+ }
for ((src, dst) <- filesToMove) {
fs.rename(new Path(src), new Path(dst))
}
- fs.delete(absPathStagingDir, true)
+
+ if (dynamicPartitionOverwrite) {
+ val partitionPaths = allPartitionPaths.foldLeft(Set[String]())(_ ++ _)
+ logDebug(s"Clean up default partition directories for overwriting: $partitionPaths")
+ for (part <- partitionPaths) {
+ val finalPartPath = new Path(path, part)
+ if (!fs.delete(finalPartPath, true) && !fs.exists(finalPartPath.getParent)) {
+ // According to the official hadoop FileSystem API spec, delete op should assume
+ // the destination is no longer present regardless of return value, thus we do not
+ // need to double check if finalPartPath exists before rename.
+ // Also in our case, based on the spec, delete returns false only when finalPartPath
+ // does not exist. When this happens, we need to take action if parent of finalPartPath
+ // also does not exist(e.g. the scenario described on SPARK-23815), because
+ // FileSystem API spec on rename op says the rename dest(finalPartPath) must have
+ // a parent that exists, otherwise we may get unexpected result on the rename.
+ fs.mkdirs(finalPartPath.getParent)
+ }
+ fs.rename(new Path(stagingDir, part), finalPartPath)
+ }
+ }
+
+ fs.delete(stagingDir, true)
}
}
override def abortJob(jobContext: JobContext): Unit = {
committer.abortJob(jobContext, JobStatus.State.FAILED)
if (hasValidPath) {
- val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
- fs.delete(absPathStagingDir, true)
+ val fs = stagingDir.getFileSystem(jobContext.getConfiguration)
+ fs.delete(stagingDir, true)
}
}
@@ -165,13 +217,14 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
committer = setupCommitter(taskContext)
committer.setupTask(taskContext)
addedAbsPathFiles = mutable.Map[String, String]()
+ partitionPaths = mutable.Set[String]()
}
override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = {
val attemptId = taskContext.getTaskAttemptID
SparkHadoopMapRedUtil.commitTask(
committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId)
- new TaskCommitMessage(addedAbsPathFiles.toMap)
+ new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet)
}
override def abortTask(taskContext: TaskAttemptContext): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
index aaae33ca4e6f..1b049b786023 100644
--- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala
@@ -67,13 +67,13 @@ private[spark] abstract class LauncherBackend {
}
def setAppId(appId: String): Unit = {
- if (connection != null) {
+ if (connection != null && isConnected) {
connection.send(new SetAppId(appId))
}
}
def setState(state: SparkAppHandle.State): Unit = {
- if (connection != null && lastState != state) {
+ if (connection != null && isConnected && lastState != state) {
connection.send(new SetState(state))
lastState = state
}
@@ -114,10 +114,10 @@ private[spark] abstract class LauncherBackend {
override def close(): Unit = {
try {
+ _isConnected = false
super.close()
} finally {
onDisconnected()
- _isConnected = false
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index c9ed12f4e1bd..13db4985b0b8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -90,7 +90,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
// Otherwise, interpolate the number of partitions we need to try, but overestimate it
// by 50%. We also cap the estimation in the end.
if (results.size == 0) {
- numPartsToTry = partsScanned * 4
+ numPartsToTry = partsScanned * 4L
} else {
// the left side of max is >=1 whenever partsScanned >= 2
numPartsToTry = Math.max(1,
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index 10451a324b0f..94e7d0b38cba 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -266,17 +266,17 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10)
numCreated += 1
}
}
- tries = 0
// if we don't have enough partition groups, create duplicates
while (numCreated < targetLen) {
- val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries)
- tries += 1
+ // Copy the preferred location from a random input partition.
+ // This helps in avoiding skew when the input partitions are clustered by preferred location.
+ val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(
+ rnd.nextInt(partitionLocs.partsWithLocs.length))
val pgroup = new PartitionGroup(Some(nxt_replica))
groupArr += pgroup
groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup
addPartToPGroup(nxt_part, pgroup)
numCreated += 1
- if (tries >= partitionLocs.partsWithLocs.length) tries = 0
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 2480559a41b7..44895abc7bd4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -17,7 +17,7 @@
package org.apache.spark.rdd
-import java.io.IOException
+import java.io.{FileNotFoundException, IOException}
import java.text.SimpleDateFormat
import java.util.{Date, Locale}
@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.mapred._
import org.apache.hadoop.mapred.lib.CombineFileSplit
import org.apache.hadoop.mapreduce.TaskType
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.hadoop.util.ReflectionUtils
import org.apache.spark._
@@ -134,6 +135,8 @@ class HadoopRDD[K, V](
private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES)
+ private val ignoreMissingFiles = sparkContext.conf.get(IGNORE_MISSING_FILES)
+
private val ignoreEmptySplits = sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS)
// Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads.
@@ -197,17 +200,24 @@ class HadoopRDD[K, V](
val jobConf = getJobConf()
// add the credentials here as this can be called before SparkContext initialized
SparkHadoopUtil.get.addCredentials(jobConf)
- val allInputSplits = getInputFormat(jobConf).getSplits(jobConf, minPartitions)
- val inputSplits = if (ignoreEmptySplits) {
- allInputSplits.filter(_.getLength > 0)
- } else {
- allInputSplits
- }
- val array = new Array[Partition](inputSplits.size)
- for (i <- 0 until inputSplits.size) {
- array(i) = new HadoopPartition(id, i, inputSplits(i))
+ try {
+ val allInputSplits = getInputFormat(jobConf).getSplits(jobConf, minPartitions)
+ val inputSplits = if (ignoreEmptySplits) {
+ allInputSplits.filter(_.getLength > 0)
+ } else {
+ allInputSplits
+ }
+ val array = new Array[Partition](inputSplits.size)
+ for (i <- 0 until inputSplits.size) {
+ array(i) = new HadoopPartition(id, i, inputSplits(i))
+ }
+ array
+ } catch {
+ case e: InvalidInputException if ignoreMissingFiles =>
+ logWarning(s"${jobConf.get(FileInputFormat.INPUT_DIR)} doesn't exist and no" +
+ s" partitions returned from this path.", e)
+ Array.empty[Partition]
}
- array
}
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
@@ -256,6 +266,12 @@ class HadoopRDD[K, V](
try {
inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
} catch {
+ case e: FileNotFoundException if ignoreMissingFiles =>
+ logWarning(s"Skipped missing file: ${split.inputSplit}", e)
+ finished = true
+ null
+ // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+ case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e: IOException if ignoreCorruptFiles =>
logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e)
finished = true
@@ -276,6 +292,11 @@ class HadoopRDD[K, V](
try {
finished = !reader.next(key, value)
} catch {
+ case e: FileNotFoundException if ignoreMissingFiles =>
+ logWarning(s"Skipped missing file: ${split.inputSplit}", e)
+ finished = true
+ // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+ case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e: IOException if ignoreCorruptFiles =>
logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e)
finished = true
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index e4dd1b6a8249..ff66a04859d1 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -17,7 +17,7 @@
package org.apache.spark.rdd
-import java.io.IOException
+import java.io.{FileNotFoundException, IOException}
import java.text.SimpleDateFormat
import java.util.{Date, Locale}
@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce._
-import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit}
+import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileInputFormat, FileSplit, InvalidInputException}
import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl}
import org.apache.spark._
@@ -90,6 +90,8 @@ class NewHadoopRDD[K, V](
private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES)
+ private val ignoreMissingFiles = sparkContext.conf.get(IGNORE_MISSING_FILES)
+
private val ignoreEmptySplits = sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS)
def getConf: Configuration = {
@@ -124,17 +126,25 @@ class NewHadoopRDD[K, V](
configurable.setConf(_conf)
case _ =>
}
- val allRowSplits = inputFormat.getSplits(new JobContextImpl(_conf, jobId)).asScala
- val rawSplits = if (ignoreEmptySplits) {
- allRowSplits.filter(_.getLength > 0)
- } else {
- allRowSplits
- }
- val result = new Array[Partition](rawSplits.size)
- for (i <- 0 until rawSplits.size) {
- result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
+ try {
+ val allRowSplits = inputFormat.getSplits(new JobContextImpl(_conf, jobId)).asScala
+ val rawSplits = if (ignoreEmptySplits) {
+ allRowSplits.filter(_.getLength > 0)
+ } else {
+ allRowSplits
+ }
+ val result = new Array[Partition](rawSplits.size)
+ for (i <- 0 until rawSplits.size) {
+ result(i) =
+ new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
+ }
+ result
+ } catch {
+ case e: InvalidInputException if ignoreMissingFiles =>
+ logWarning(s"${_conf.get(FileInputFormat.INPUT_DIR)} doesn't exist and no" +
+ s" partitions returned from this path.", e)
+ Array.empty[Partition]
}
- result
}
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
@@ -189,6 +199,12 @@ class NewHadoopRDD[K, V](
_reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
_reader
} catch {
+ case e: FileNotFoundException if ignoreMissingFiles =>
+ logWarning(s"Skipped missing file: ${split.serializableHadoopSplit}", e)
+ finished = true
+ null
+ // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+ case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e: IOException if ignoreCorruptFiles =>
logWarning(
s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}",
@@ -213,6 +229,11 @@ class NewHadoopRDD[K, V](
try {
finished = !reader.nextKeyValue
} catch {
+ case e: FileNotFoundException if ignoreMissingFiles =>
+ logWarning(s"Skipped missing file: ${split.serializableHadoopSplit}", e)
+ finished = true
+ // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+ case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e: IOException if ignoreCorruptFiles =>
logWarning(
s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}",
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 8798dfc92536..0574abdca32a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -150,7 +150,7 @@ abstract class RDD[T: ClassTag](
val id: Int = sc.newRddId()
/** A friendly name for this RDD */
- @transient var name: String = null
+ @transient var name: String = _
/** Assign a name to this RDD */
def setName(_name: String): this.type = {
@@ -224,8 +224,8 @@ abstract class RDD[T: ClassTag](
// Our dependencies and partitions will be gotten by calling subclass's methods below, and will
// be overwritten when we're checkpointed
- private var dependencies_ : Seq[Dependency[_]] = null
- @transient private var partitions_ : Array[Partition] = null
+ private var dependencies_ : Seq[Dependency[_]] = _
+ @transient private var partitions_ : Array[Partition] = _
/** An Option holding our checkpoint RDD, if we are checkpointed */
private def checkpointRDD: Option[CheckpointRDD[T]] = checkpointData.flatMap(_.checkpointRDD)
@@ -297,7 +297,7 @@ abstract class RDD[T: ClassTag](
private[spark] def getNarrowAncestors: Seq[RDD[_]] = {
val ancestors = new mutable.HashSet[RDD[_]]
- def visit(rdd: RDD[_]) {
+ def visit(rdd: RDD[_]): Unit = {
val narrowDependencies = rdd.dependencies.filter(_.isInstanceOf[NarrowDependency[_]])
val narrowParents = narrowDependencies.map(_.rdd)
val narrowParentsNotVisited = narrowParents.filterNot(ancestors.contains)
@@ -414,6 +414,8 @@ abstract class RDD[T: ClassTag](
*
* If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
* which can avoid performing a shuffle.
+ *
+ * TODO Fix the Shuffle+Repartition data loss issue described in SPARK-23207.
*/
def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
coalesce(numPartitions, shuffle = true)
@@ -449,7 +451,7 @@ abstract class RDD[T: ClassTag](
if (shuffle) {
/** Distributes elements evenly across output partitions, starting from a random partition. */
val distributePartition = (index: Int, items: Iterator[T]) => {
- var position = (new Random(hashing.byteswap32(index))).nextInt(numPartitions)
+ var position = new Random(hashing.byteswap32(index)).nextInt(numPartitions)
items.map { t =>
// Note that the hash code of the key will just be the key itself. The HashPartitioner
// will mod it with the number of total partitions.
@@ -951,7 +953,7 @@ abstract class RDD[T: ClassTag](
def collectPartition(p: Int): Array[T] = {
sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p)).head
}
- (0 until partitions.length).iterator.flatMap(i => collectPartition(i))
+ partitions.indices.iterator.flatMap(i => collectPartition(i))
}
/**
@@ -1338,6 +1340,7 @@ abstract class RDD[T: ClassTag](
// The number of partitions to try in this iteration. It is ok for this number to be
// greater than totalParts because we actually cap it at totalParts in runJob.
var numPartsToTry = 1L
+ val left = num - buf.size
if (partsScanned > 0) {
// If we didn't find any rows after the previous iteration, quadruple and retry.
// Otherwise, interpolate the number of partitions we need to try, but overestimate
@@ -1345,13 +1348,12 @@ abstract class RDD[T: ClassTag](
if (buf.isEmpty) {
numPartsToTry = partsScanned * scaleUpFactor
} else {
- // the left side of max is >=1 whenever partsScanned >= 2
- numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1)
+ // As left > 0, numPartsToTry is always >= 1
+ numPartsToTry = Math.ceil(1.5 * left * partsScanned / buf.size).toInt
numPartsToTry = Math.min(numPartsToTry, partsScanned * scaleUpFactor)
}
}
- val left = num - buf.size
val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt)
val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p)
@@ -1677,8 +1679,7 @@ abstract class RDD[T: ClassTag](
// an RDD and its parent in every batch, in which case the parent may never be checkpointed
// and its lineage never truncated, leading to OOMs in the long run (SPARK-6847).
private val checkpointAllMarkedAncestors =
- Option(sc.getLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS))
- .map(_.toBoolean).getOrElse(false)
+ Option(sc.getLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS)).exists(_.toBoolean)
/** Returns the first parent RDD */
protected[spark] def firstParent[U: ClassTag]: RDD[U] = {
@@ -1686,7 +1687,7 @@ abstract class RDD[T: ClassTag](
}
/** Returns the jth parent RDD: e.g. rdd.parent[T](0) is equivalent to rdd.firstParent[T] */
- protected[spark] def parent[U: ClassTag](j: Int) = {
+ protected[spark] def parent[U: ClassTag](j: Int): RDD[U] = {
dependencies(j).rdd.asInstanceOf[RDD[U]]
}
@@ -1754,7 +1755,7 @@ abstract class RDD[T: ClassTag](
* collected. Subclasses of RDD may override this method for implementing their own cleaning
* logic. See [[org.apache.spark.rdd.UnionRDD]] for an example.
*/
- protected def clearDependencies() {
+ protected def clearDependencies(): Unit = {
dependencies_ = null
}
@@ -1790,7 +1791,7 @@ abstract class RDD[T: ClassTag](
val lastDepStrings =
debugString(lastDep.rdd, prefix, lastDep.isInstanceOf[ShuffleDependency[_, _, _]], true)
- (frontDepStrings ++ lastDepStrings)
+ frontDepStrings ++ lastDepStrings
}
}
// The first RDD in the dependency stack has no parents, so no need for a +-
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
index f951591e02a5..a2936d6ad539 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
@@ -332,16 +332,14 @@ private[netty] class NettyRpcEnv(
val pipe = Pipe.open()
val source = new FileDownloadChannel(pipe.source())
- try {
+ Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
val client = downloadClient(parsedUri.getHost(), parsedUri.getPort())
val callback = new FileDownloadCallback(pipe.sink(), source, client)
client.stream(parsedUri.getPath(), callback)
- } catch {
- case e: Exception =>
- pipe.sink().close()
- source.close()
- throw e
- }
+ })(catchBlock = {
+ pipe.sink().close()
+ source.close()
+ })
source
}
@@ -370,24 +368,33 @@ private[netty] class NettyRpcEnv(
fileDownloadFactory.createClient(host, port)
}
- private class FileDownloadChannel(source: ReadableByteChannel) extends ReadableByteChannel {
+ private class FileDownloadChannel(source: Pipe.SourceChannel) extends ReadableByteChannel {
@volatile private var error: Throwable = _
def setError(e: Throwable): Unit = {
+ // This setError callback is invoked by internal RPC threads in order to propagate remote
+ // exceptions to application-level threads which are reading from this channel. When an
+ // RPC error occurs, the RPC system will call setError() and then will close the
+ // Pipe.SinkChannel corresponding to the other end of the `source` pipe. Closing of the pipe
+ // sink will cause `source.read()` operations to return EOF, unblocking the application-level
+ // reading thread. Thus there is no need to actually call `source.close()` here in the
+ // onError() callback and, in fact, calling it here would be dangerous because the close()
+ // would be asynchronous with respect to the read() call and could trigger race-conditions
+ // that lead to data corruption. See the PR for SPARK-22982 for more details on this topic.
error = e
- source.close()
}
override def read(dst: ByteBuffer): Int = {
Try(source.read(dst)) match {
+ // See the documentation above in setError(): if an RPC error has occurred then setError()
+ // will be called to propagate the RPC error and then `source`'s corresponding
+ // Pipe.SinkChannel will be closed, unblocking this read. In that case, we want to propagate
+ // the remote RPC exception (and not any exceptions triggered by the pipe close, such as
+ // ChannelClosedException), hence this `error != null` check:
+ case _ if error != null => throw error
case Success(bytesRead) => bytesRead
- case Failure(readErr) =>
- if (error != null) {
- throw error
- } else {
- throw readErr
- }
+ case Failure(readErr) => throw readErr
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
index 7e14938acd8e..c1fedd63f6a9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
@@ -166,7 +166,7 @@ private class AsyncEventQueue(val name: String, conf: SparkConf, metrics: LiveLi
val prevLastReportTimestamp = lastReportTimestamp
lastReportTimestamp = System.currentTimeMillis()
val previous = new java.util.Date(prevLastReportTimestamp)
- logWarning(s"Dropped $droppedEvents events from $name since $previous.")
+ logWarning(s"Dropped $droppedCount events from $name since $previous.")
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
index cd8e61d6d020..30cf75d43ee0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
@@ -152,7 +152,8 @@ private[scheduler] class BlacklistTracker (
case Some(a) =>
logInfo(s"Killing blacklisted executor id $exec " +
s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
- a.killExecutors(Seq(exec), true, true)
+ a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
+ force = true)
case None =>
logWarning(s"Not attempting to kill blacklisted executor id $exec " +
s"since allocation client is not defined.")
@@ -209,7 +210,7 @@ private[scheduler] class BlacklistTracker (
updateNextExpiryTime()
killBlacklistedExecutor(exec)
- val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
+ val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(host, HashSet[String]())
blacklistedExecsOnNode += exec
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index c2498d4808e9..78b6b34b5d2b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.internal.config
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDD, RDDCheckpointData}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.storage._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -815,7 +815,8 @@ class DAGScheduler(
private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) {
// Note that there is a chance that this task is launched after the stage is cancelled.
// In that case, we wouldn't have the stage anymore in stageIdToStage.
- val stageAttemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1)
+ val stageAttemptId =
+ stageIdToStage.get(task.stageId).map(_.latestInfo.attemptNumber).getOrElse(-1)
listenerBus.post(SparkListenerTaskStart(task.stageId, stageAttemptId, taskInfo))
}
@@ -1015,15 +1016,24 @@ class DAGScheduler(
// might modify state of objects referenced in their closures. This is necessary in Hadoop
// where the JobConf/Configuration object is not thread-safe.
var taskBinary: Broadcast[Array[Byte]] = null
+ var partitions: Array[Partition] = null
try {
// For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
// For ResultTask, serialize and broadcast (rdd, func).
- val taskBinaryBytes: Array[Byte] = stage match {
- case stage: ShuffleMapStage =>
- JavaUtils.bufferToArray(
- closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
- case stage: ResultStage =>
- JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef))
+ var taskBinaryBytes: Array[Byte] = null
+ // taskBinaryBytes and partitions are both effected by the checkpoint status. We need
+ // this synchronization in case another concurrent job is checkpointing this RDD, so we get a
+ // consistent view of both variables.
+ RDDCheckpointData.synchronized {
+ taskBinaryBytes = stage match {
+ case stage: ShuffleMapStage =>
+ JavaUtils.bufferToArray(
+ closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
+ case stage: ResultStage =>
+ JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef))
+ }
+
+ partitions = stage.rdd.partitions
}
taskBinary = sc.broadcast(taskBinaryBytes)
@@ -1048,9 +1058,9 @@ class DAGScheduler(
stage.pendingPartitions.clear()
partitionsToCompute.map { id =>
val locs = taskIdToLocations(id)
- val part = stage.rdd.partitions(id)
+ val part = partitions(id)
stage.pendingPartitions += id
- new ShuffleMapTask(stage.id, stage.latestInfo.attemptId,
+ new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId),
Option(sc.applicationId), sc.applicationAttemptId)
}
@@ -1058,9 +1068,9 @@ class DAGScheduler(
case stage: ResultStage =>
partitionsToCompute.map { id =>
val p: Int = stage.partitions(id)
- val part = stage.rdd.partitions(p)
+ val part = partitions(p)
val locs = taskIdToLocations(id)
- new ResultTask(stage.id, stage.latestInfo.attemptId,
+ new ResultTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, id, properties, serializedTaskMetrics,
Option(jobId), Option(sc.applicationId), sc.applicationAttemptId)
}
@@ -1076,23 +1086,22 @@ class DAGScheduler(
logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " +
s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})")
taskScheduler.submitTasks(new TaskSet(
- tasks.toArray, stage.id, stage.latestInfo.attemptId, jobId, properties))
+ tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties))
} else {
// Because we posted SparkListenerStageSubmitted earlier, we should mark
// the stage as completed here in case there are no tasks to run
markStageAsFinished(stage, None)
- val debugString = stage match {
+ stage match {
case stage: ShuffleMapStage =>
- s"Stage ${stage} is actually done; " +
- s"(available: ${stage.isAvailable}," +
- s"available outputs: ${stage.numAvailableOutputs}," +
- s"partitions: ${stage.numPartitions})"
+ logDebug(s"Stage ${stage} is actually done; " +
+ s"(available: ${stage.isAvailable}," +
+ s"available outputs: ${stage.numAvailableOutputs}," +
+ s"partitions: ${stage.numPartitions})")
+ markMapStageJobsAsFinished(stage)
case stage : ResultStage =>
- s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})"
+ logDebug(s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})")
}
- logDebug(debugString)
-
submitWaitingChildStages(stage)
}
}
@@ -1245,7 +1254,7 @@ class DAGScheduler(
val status = event.result.asInstanceOf[MapStatus]
val execId = status.location.executorId
logDebug("ShuffleMapTask finished on " + execId)
- if (stageIdToStage(task.stageId).latestInfo.attemptId == task.stageAttemptId) {
+ if (stageIdToStage(task.stageId).latestInfo.attemptNumber == task.stageAttemptId) {
// This task was for the currently running attempt of the stage. Since the task
// completed successfully from the perspective of the TaskSetManager, mark it as
// no longer pending (the TaskSetManager may consider the task complete even
@@ -1297,13 +1306,7 @@ class DAGScheduler(
shuffleStage.findMissingPartitions().mkString(", "))
submitStage(shuffleStage)
} else {
- // Mark any map-stage jobs waiting on this stage as finished
- if (shuffleStage.mapStageJobs.nonEmpty) {
- val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep)
- for (job <- shuffleStage.mapStageJobs) {
- markMapStageJobAsFinished(job, stats)
- }
- }
+ markMapStageJobsAsFinished(shuffleStage)
submitWaitingChildStages(shuffleStage)
}
}
@@ -1324,10 +1327,10 @@ class DAGScheduler(
val failedStage = stageIdToStage(task.stageId)
val mapStage = shuffleIdToMapStage(shuffleId)
- if (failedStage.latestInfo.attemptId != task.stageAttemptId) {
+ if (failedStage.latestInfo.attemptNumber != task.stageAttemptId) {
logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" +
s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
- s"(attempt ID ${failedStage.latestInfo.attemptId}) running")
+ s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
} else {
// It is likely that we receive multiple FetchFailed for a single stage (because we have
// multiple tasks running concurrently on different executors). In that case, it is
@@ -1423,6 +1426,16 @@ class DAGScheduler(
}
}
+ private[scheduler] def markMapStageJobsAsFinished(shuffleStage: ShuffleMapStage): Unit = {
+ // Mark any map-stage jobs waiting on this stage as finished
+ if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) {
+ val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep)
+ for (job <- shuffleStage.mapStageJobs) {
+ markMapStageJobAsFinished(job, stats)
+ }
+ }
+ }
+
/**
* Responds to an executor being lost. This is called inside the event loop, so it assumes it can
* modify the scheduler's internal state. Use executorLost() to post a loss event from outside.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index b3a5b1f1e05b..69bc51c1ecf9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -207,6 +207,15 @@ private[spark] class EventLoggingListener(
logEvent(event, flushLogger = true)
}
+ override def onExecutorBlacklistedForStage(
+ event: SparkListenerExecutorBlacklistedForStage): Unit = {
+ logEvent(event, flushLogger = true)
+ }
+
+ override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = {
+ logEvent(event, flushLogger = true)
+ }
+
override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = {
logEvent(event, flushLogger = true)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
index 23121402b102..ba6387a8f08a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
@@ -62,6 +62,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
private val queues = new CopyOnWriteArrayList[AsyncEventQueue]()
+ // Visible for testing.
+ @volatile private[scheduler] var queuedEvents = new mutable.ListBuffer[SparkListenerEvent]()
+
/** Add a listener to queue shared by all non-internal listeners. */
def addToSharedQueue(listener: SparkListenerInterface): Unit = {
addToQueue(listener, SHARED_QUEUE)
@@ -125,13 +128,39 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
/** Post an event to all queues. */
def post(event: SparkListenerEvent): Unit = {
- if (!stopped.get()) {
- metrics.numEventsPosted.inc()
- val it = queues.iterator()
- while (it.hasNext()) {
- it.next().post(event)
+ if (stopped.get()) {
+ return
+ }
+
+ metrics.numEventsPosted.inc()
+
+ // If the event buffer is null, it means the bus has been started and we can avoid
+ // synchronization and post events directly to the queues. This should be the most
+ // common case during the life of the bus.
+ if (queuedEvents == null) {
+ postToQueues(event)
+ return
+ }
+
+ // Otherwise, need to synchronize to check whether the bus is started, to make sure the thread
+ // calling start() picks up the new event.
+ synchronized {
+ if (!started.get()) {
+ queuedEvents += event
+ return
}
}
+
+ // If the bus was already started when the check above was made, just post directly to the
+ // queues.
+ postToQueues(event)
+ }
+
+ private def postToQueues(event: SparkListenerEvent): Unit = {
+ val it = queues.iterator()
+ while (it.hasNext()) {
+ it.next().post(event)
+ }
}
/**
@@ -149,7 +178,11 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
}
this.sparkContext = sc
- queues.asScala.foreach(_.start(sc))
+ queues.asScala.foreach { q =>
+ q.start(sc)
+ queuedEvents.foreach(q.post)
+ }
+ queuedEvents = null
metricsSystem.registerSource(metrics)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
index c9cd662f5709..226c23733c87 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
@@ -115,6 +115,8 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging {
}
}
} catch {
+ case e: HaltReplayException =>
+ // Just stop replay.
case _: EOFException if maybeTruncated =>
case ioe: IOException =>
throw ioe
@@ -124,8 +126,17 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging {
}
}
+ override protected def isIgnorableException(e: Throwable): Boolean = {
+ e.isInstanceOf[HaltReplayException]
+ }
+
}
+/**
+ * Exception that can be thrown by listeners to halt replay. This is handled by ReplayListenerBus
+ * only, and will cause errors if thrown when using other bus implementations.
+ */
+private[spark] class HaltReplayException extends RuntimeException
private[spark] object ReplayListenerBus {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 3b677ca9657d..8a112f6a37b9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -120,6 +120,24 @@ case class SparkListenerExecutorBlacklisted(
taskFailures: Int)
extends SparkListenerEvent
+@DeveloperApi
+case class SparkListenerExecutorBlacklistedForStage(
+ time: Long,
+ executorId: String,
+ taskFailures: Int,
+ stageId: Int,
+ stageAttemptId: Int)
+ extends SparkListenerEvent
+
+@DeveloperApi
+case class SparkListenerNodeBlacklistedForStage(
+ time: Long,
+ hostId: String,
+ executorFailures: Int,
+ stageId: Int,
+ stageAttemptId: Int)
+ extends SparkListenerEvent
+
@DeveloperApi
case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String)
extends SparkListenerEvent
@@ -261,6 +279,17 @@ private[spark] trait SparkListenerInterface {
*/
def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit
+ /**
+ * Called when the driver blacklists an executor for a stage.
+ */
+ def onExecutorBlacklistedForStage(
+ executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit
+
+ /**
+ * Called when the driver blacklists a node for a stage.
+ */
+ def onNodeBlacklistedForStage(nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit
+
/**
* Called when the driver re-enables a previously blacklisted executor.
*/
@@ -339,6 +368,12 @@ abstract class SparkListener extends SparkListenerInterface {
override def onExecutorBlacklisted(
executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { }
+ def onExecutorBlacklistedForStage(
+ executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit = { }
+
+ def onNodeBlacklistedForStage(
+ nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit = { }
+
override def onExecutorUnblacklisted(
executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index 056c0cbded43..ff19cc65552e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus
listener.onExecutorAdded(executorAdded)
case executorRemoved: SparkListenerExecutorRemoved =>
listener.onExecutorRemoved(executorRemoved)
+ case executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage =>
+ listener.onExecutorBlacklistedForStage(executorBlacklistedForStage)
+ case nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage =>
+ listener.onNodeBlacklistedForStage(nodeBlacklistedForStage)
case executorBlacklisted: SparkListenerExecutorBlacklisted =>
listener.onExecutorBlacklisted(executorBlacklisted)
case executorUnblacklisted: SparkListenerExecutorUnblacklisted =>
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index c513ed36d168..903e25b7986f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -30,7 +30,7 @@ import org.apache.spark.storage.RDDInfo
@DeveloperApi
class StageInfo(
val stageId: Int,
- val attemptId: Int,
+ @deprecated("Use attemptNumber instead", "2.3.0") val attemptId: Int,
val name: String,
val numTasks: Int,
val rddInfos: Seq[RDDInfo],
@@ -56,6 +56,8 @@ class StageInfo(
completionTime = Some(System.currentTimeMillis)
}
+ def attemptNumber(): Int = attemptId
+
private[spark] def getStatusString: String = {
if (completionTime.isDefined) {
if (failureReason.isDefined) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
index 3c8cab7504c1..3c7af4f6146f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
@@ -79,7 +79,7 @@ class StatsReportListener extends SparkListener with Logging {
x => info.completionTime.getOrElse(System.currentTimeMillis()) - x
).getOrElse("-")
- s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " +
+ s"Stage(${info.stageId}, ${info.attemptNumber}); Name: '${info.name}'; " +
s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " +
s"Took: $timeTaken msec"
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 7767ef1803a0..f536fc2a5f0a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -79,6 +79,7 @@ private[spark] abstract class Task[T](
SparkEnv.get.blockManager.registerTask(taskAttemptId)
context = new TaskContextImpl(
stageId,
+ stageAttemptId, // stageAttemptId and stageAttemptNumber are semantically equal
partitionId,
taskAttemptId,
attemptNumber,
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 0c11806b3981..8e97b3da3382 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -689,6 +689,20 @@ private[spark] class TaskSchedulerImpl(
}
}
+ /**
+ * Marks the task has completed in all TaskSetManagers for the given stage.
+ *
+ * After stage failure and retry, there may be multiple TaskSetManagers for the stage.
+ * If an earlier attempt of a stage completes a task, we should ensure that the later attempts
+ * do not also submit those same tasks. That also means that a task completion from an earlier
+ * attempt can lead to the entire stage getting marked as successful.
+ */
+ private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, partitionId: Int) = {
+ taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm =>
+ tsm.markPartitionCompleted(partitionId)
+ }
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
index 233781f3d971..b680979a466a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
@@ -36,8 +36,12 @@ import org.apache.spark.util.Clock
* [[TaskSetManager]] this class is designed only to be called from code with a lock on the
* TaskScheduler (e.g. its event handlers). It should not be called from other threads.
*/
-private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, val clock: Clock)
- extends Logging {
+private[scheduler] class TaskSetBlacklist(
+ private val listenerBus: LiveListenerBus,
+ val conf: SparkConf,
+ val stageId: Int,
+ val stageAttemptId: Int,
+ val clock: Clock) extends Logging {
private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR)
private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE)
@@ -128,16 +132,23 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
}
// Check if enough tasks have failed on the executor to blacklist it for the entire stage.
- if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) {
+ val numFailures = execFailures.numUniqueTasksWithFailures
+ if (numFailures >= MAX_FAILURES_PER_EXEC_STAGE) {
if (blacklistedExecs.add(exec)) {
logInfo(s"Blacklisting executor ${exec} for stage $stageId")
// This executor has been pushed into the blacklist for this stage. Let's check if it
// pushes the whole node into the blacklist.
val blacklistedExecutorsOnNode =
execsWithFailuresOnNode.filter(blacklistedExecs.contains(_))
- if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) {
+ val now = clock.getTimeMillis()
+ listenerBus.post(
+ SparkListenerExecutorBlacklistedForStage(now, exec, numFailures, stageId, stageAttemptId))
+ val numFailExec = blacklistedExecutorsOnNode.size
+ if (numFailExec >= MAX_FAILED_EXEC_PER_NODE_STAGE) {
if (blacklistedNodes.add(host)) {
logInfo(s"Blacklisting ${host} for stage $stageId")
+ listenerBus.post(
+ SparkListenerNodeBlacklistedForStage(now, host, numFailExec, stageId, stageAttemptId))
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index c3ed11bfe352..195fc8025e4b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -64,8 +64,7 @@ private[spark] class TaskSetManager(
val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75)
val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5)
- // Limit of bytes for total size of results (default is 1GB)
- val maxResultSize = Utils.getMaxResultSize(conf)
+ val maxResultSize = conf.get(config.MAX_RESULT_SIZE)
val speculationEnabled = conf.getBoolean("spark.speculation", false)
@@ -74,6 +73,8 @@ private[spark] class TaskSetManager(
val ser = env.closureSerializer.newInstance()
val tasks = taskSet.tasks
+ private[scheduler] val partitionToIndex = tasks.zipWithIndex
+ .map { case (t, idx) => t.partitionId -> idx }.toMap
val numTasks = tasks.length
val copiesRunning = new Array[Int](numTasks)
@@ -102,7 +103,7 @@ private[spark] class TaskSetManager(
private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = {
blacklistTracker.map { _ =>
- new TaskSetBlacklist(conf, stageId, clock)
+ new TaskSetBlacklist(sched.sc.listenerBus, conf, stageId, taskSet.stageAttemptId, clock)
}
}
@@ -154,7 +155,7 @@ private[spark] class TaskSetManager(
private[scheduler] val speculatableTasks = new HashSet[Int]
// Task index, start and finish time for each task attempt (indexed by task ID)
- private val taskInfos = new HashMap[Long, TaskInfo]
+ private[scheduler] val taskInfos = new HashMap[Long, TaskInfo]
// Use a MedianHeap to record durations of successful tasks so we know when to launch
// speculative tasks. This is only used when speculation is enabled, to avoid the overhead
@@ -288,7 +289,7 @@ private[spark] class TaskSetManager(
None
}
- /** Check whether a task is currently running an attempt on a given host */
+ /** Check whether a task once ran an attempt on a given host */
private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = {
taskAttempts(taskIndex).exists(_.host == host)
}
@@ -755,6 +756,9 @@ private[spark] class TaskSetManager(
logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id +
" because task " + index + " has already completed successfully")
}
+ // There may be multiple tasksets for this stage -- we let all of them know that the partition
+ // was completed. This may result in some of the tasksets getting completed.
+ sched.markPartitionCompletedInAllTaskSets(stageId, tasks(index).partitionId)
// This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the
// "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not
// "deserialize" the value when holding a lock to avoid blocking other threads. So we call
@@ -765,6 +769,19 @@ private[spark] class TaskSetManager(
maybeFinishTaskSet()
}
+ private[scheduler] def markPartitionCompleted(partitionId: Int): Unit = {
+ partitionToIndex.get(partitionId).foreach { index =>
+ if (!successful(index)) {
+ tasksSuccessful += 1
+ successful(index) = true
+ if (tasksSuccessful == numTasks) {
+ isZombie = true
+ }
+ maybeFinishTaskSet()
+ }
+ }
+ }
+
/**
* Marks the task as failed, re-adds it to the list of pending tasks, and notifies the
* DAG Scheduler.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 4d75063fbf1c..5627a557a12f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -147,7 +147,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
case KillExecutorsOnHost(host) =>
scheduler.getExecutorsAliveOnHost(host).foreach { exec =>
- killExecutors(exec.toSeq, replace = true, force = true)
+ killExecutors(exec.toSeq, adjustTargetNumExecutors = false, countFailures = false,
+ force = true)
}
case UpdateDelegationTokens(newDelegationTokens) =>
@@ -584,18 +585,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
/**
* Request that the cluster manager kill the specified executors.
*
- * When asking the executor to be replaced, the executor loss is considered a failure, and
- * killed tasks that are running on the executor will count towards the failure limits. If no
- * replacement is being requested, then the tasks will not count towards the limit.
- *
* @param executorIds identifiers of executors to kill
- * @param replace whether to replace the killed executors with new ones, default false
+ * @param adjustTargetNumExecutors whether the target number of executors be adjusted down
+ * after these executors have been killed
+ * @param countFailures if there are tasks running on the executors when they are killed, whether
+ * those failures be counted to task failure limits?
* @param force whether to force kill busy executors, default false
* @return the ids of the executors acknowledged by the cluster manager to be removed.
*/
final override def killExecutors(
executorIds: Seq[String],
- replace: Boolean,
+ adjustTargetNumExecutors: Boolean,
+ countFailures: Boolean,
force: Boolean): Seq[String] = {
logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}")
@@ -610,7 +611,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
val executorsToKill = knownExecutors
.filter { id => !executorsPendingToRemove.contains(id) }
.filter { id => force || !scheduler.isExecutorBusy(id) }
- executorsToKill.foreach { id => executorsPendingToRemove(id) = !replace }
+ executorsToKill.foreach { id => executorsPendingToRemove(id) = !countFailures }
logInfo(s"Actual list of executor(s) to be killed is ${executorsToKill.mkString(", ")}")
@@ -618,12 +619,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// with the cluster manager to avoid allocating new ones. When computing the new target,
// take into account executors that are pending to be added or removed.
val adjustTotalExecutors =
- if (!replace) {
+ if (adjustTargetNumExecutors) {
requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0)
if (requestedTotalExecutors !=
(numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) {
logDebug(
- s"""killExecutors($executorIds, $replace, $force): Executor counts do not match:
+ s"""killExecutors($executorIds, $adjustTargetNumExecutors, $countFailures, $force):
+ |Executor counts do not match:
|requestedTotalExecutors = $requestedTotalExecutors
|numExistingExecutors = $numExistingExecutors
|numPendingExecutors = $numPendingExecutors
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 538ae05e4eea..72427dd6ce4d 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -206,6 +206,7 @@ class KryoSerializer(conf: SparkConf)
kryo.register(clazz)
} catch {
case NonFatal(_) => // do nothing
+ case _: NoClassDefFoundError if Utils.isTesting => // See SPARK-23422.
}
}
diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
index 0562d45ff57c..4103dfb10175 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -90,12 +90,11 @@ private[spark] class BlockStoreShuffleReader[K, C](
dep.aggregator.get.combineValuesByKey(keyValuesIterator, context)
}
} else {
- require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!")
interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]]
}
// Sort the output if there is a sort ordering defined.
- dep.keyOrdering match {
+ val resultIter = dep.keyOrdering match {
case Some(keyOrd: Ordering[K]) =>
// Create an ExternalSorter to sort the data.
val sorter =
@@ -104,9 +103,21 @@ private[spark] class BlockStoreShuffleReader[K, C](
context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled)
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes)
+ // Use completion callback to stop sorter if task was finished/cancelled.
+ context.addTaskCompletionListener(_ => {
+ sorter.stop()
+ })
CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop())
case None =>
aggregatedIter
}
+
+ resultIter match {
+ case _: InterruptibleIterator[Product2[K, C]] => resultIter
+ case _ =>
+ // Use another interruptible iterator here to support task cancellation as aggregator
+ // or(and) sorter may have consumed previous interruptible iterator.
+ new InterruptibleIterator[Product2[K, C]](context, resultIter)
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
index 15540485170d..d3f1c7ec1bbe 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
@@ -18,8 +18,8 @@
package org.apache.spark.shuffle
import java.io._
-
-import com.google.common.io.ByteStreams
+import java.nio.channels.Channels
+import java.nio.file.Files
import org.apache.spark.{SparkConf, SparkEnv}
import org.apache.spark.internal.Logging
@@ -84,7 +84,7 @@ private[spark] class IndexShuffleBlockResolver(
*/
private def checkIndexAndDataFile(index: File, data: File, blocks: Int): Array[Long] = {
// the index file should have `block + 1` longs as offset.
- if (index.length() != (blocks + 1) * 8) {
+ if (index.length() != (blocks + 1) * 8L) {
return null
}
val lengths = new Array[Long](blocks)
@@ -141,19 +141,6 @@ private[spark] class IndexShuffleBlockResolver(
val indexFile = getIndexFile(shuffleId, mapId)
val indexTmp = Utils.tempFileWith(indexFile)
try {
- val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp)))
- Utils.tryWithSafeFinally {
- // We take in lengths of each block, need to convert it to offsets.
- var offset = 0L
- out.writeLong(offset)
- for (length <- lengths) {
- offset += length
- out.writeLong(offset)
- }
- } {
- out.close()
- }
-
val dataFile = getDataFile(shuffleId, mapId)
// There is only one IndexShuffleBlockResolver per executor, this synchronization make sure
// the following check and rename are atomic.
@@ -166,10 +153,22 @@ private[spark] class IndexShuffleBlockResolver(
if (dataTmp != null && dataTmp.exists()) {
dataTmp.delete()
}
- indexTmp.delete()
} else {
// This is the first successful attempt in writing the map outputs for this task,
// so override any existing index and data files with the ones we wrote.
+ val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp)))
+ Utils.tryWithSafeFinally {
+ // We take in lengths of each block, need to convert it to offsets.
+ var offset = 0L
+ out.writeLong(offset)
+ for (length <- lengths) {
+ offset += length
+ out.writeLong(offset)
+ }
+ } {
+ out.close()
+ }
+
if (indexFile.exists()) {
indexFile.delete()
}
@@ -196,11 +195,24 @@ private[spark] class IndexShuffleBlockResolver(
// find out the consolidated file, then the offset within that from our index
val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId)
- val in = new DataInputStream(new FileInputStream(indexFile))
+ // SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code
+ // which is incorrectly using our file descriptor then this code will fetch the wrong offsets
+ // (which may cause a reducer to be sent a different reducer's data). The explicit position
+ // checks added here were a useful debugging aid during SPARK-22982 and may help prevent this
+ // class of issue from re-occurring in the future which is why they are left here even though
+ // SPARK-22982 is fixed.
+ val channel = Files.newByteChannel(indexFile.toPath)
+ channel.position(blockId.reduceId * 8L)
+ val in = new DataInputStream(Channels.newInputStream(channel))
try {
- ByteStreams.skipFully(in, blockId.reduceId * 8)
val offset = in.readLong()
val nextOffset = in.readLong()
+ val actualPosition = channel.position()
+ val expectedPosition = blockId.reduceId * 8L + 16
+ if (actualPosition != expectedPosition) {
+ throw new Exception(s"SPARK-22982: Incorrect channel position after index file reads: " +
+ s"expected $expectedPosition but actual position was $actualPosition.")
+ }
new FileSegmentManagedBuffer(
transportConf,
getDataFile(blockId.shuffleId, blockId.mapId),
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
index bfb4dc698e32..d9fad64f34c7 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
@@ -188,9 +188,9 @@ private[spark] object SortShuffleManager extends Logging {
log.debug(s"Can't use serialized shuffle for shuffle $shufId because the serializer, " +
s"${dependency.serializer.getClass.getName}, does not support object relocation")
false
- } else if (dependency.aggregator.isDefined) {
- log.debug(
- s"Can't use serialized shuffle for shuffle $shufId because an aggregator is defined")
+ } else if (dependency.mapSideCombine) {
+ log.debug(s"Can't use serialized shuffle for shuffle $shufId because we need to do " +
+ s"map-side aggregation")
false
} else if (numPartitions > MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE) {
log.debug(s"Can't use serialized shuffle for shuffle $shufId because it has more than " +
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
index 636b88e792bf..274399b9cc1f 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
@@ -50,7 +50,6 @@ private[spark] class SortShuffleWriter[K, V, C](
/** Write a bunch of records to this task's output */
override def write(records: Iterator[Product2[K, V]]): Unit = {
sorter = if (dep.mapSideCombine) {
- require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!")
new ExternalSorter[K, V, C](
context, dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer)
} else {
@@ -107,7 +106,6 @@ private[spark] object SortShuffleWriter {
def shouldBypassMergeSort(conf: SparkConf, dep: ShuffleDependency[_, _, _]): Boolean = {
// We cannot bypass sorting if we need to do map-side aggregation.
if (dep.mapSideCombine) {
- require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!")
false
} else {
val bypassMergeThreshold: Int = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200)
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index 487a782e865e..5ea161cd0d15 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -211,6 +211,46 @@ private[spark] class AppStatusListener(
updateBlackListStatus(event.executorId, true)
}
+ override def onExecutorBlacklistedForStage(
+ event: SparkListenerExecutorBlacklistedForStage): Unit = {
+ val now = System.nanoTime()
+
+ Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
+ setStageBlackListStatus(stage, now, event.executorId)
+ }
+ liveExecutors.get(event.executorId).foreach { exec =>
+ addBlackListedStageTo(exec, event.stageId, now)
+ }
+ }
+
+ override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = {
+ val now = System.nanoTime()
+
+ // Implicitly blacklist every available executor for the stage associated with this node
+ Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
+ val executorIds = liveExecutors.values.filter(_.host == event.hostId).map(_.executorId).toSeq
+ setStageBlackListStatus(stage, now, executorIds: _*)
+ }
+ liveExecutors.values.filter(_.hostname == event.hostId).foreach { exec =>
+ addBlackListedStageTo(exec, event.stageId, now)
+ }
+ }
+
+ private def addBlackListedStageTo(exec: LiveExecutor, stageId: Int, now: Long): Unit = {
+ exec.blacklistedInStages += stageId
+ liveUpdate(exec, now)
+ }
+
+ private def setStageBlackListStatus(stage: LiveStage, now: Long, executorIds: String*): Unit = {
+ executorIds.foreach { executorId =>
+ val executorStageSummary = stage.executorSummary(executorId)
+ executorStageSummary.isBlacklisted = true
+ maybeUpdate(executorStageSummary, now)
+ }
+ stage.blackListedExecutors ++= executorIds
+ maybeUpdate(stage, now)
+ }
+
override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = {
updateBlackListStatus(event.executorId, false)
}
@@ -377,6 +417,10 @@ private[spark] class AppStatusListener(
Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
stage.activeTasks += 1
stage.firstLaunchTime = math.min(stage.firstLaunchTime, event.taskInfo.launchTime)
+
+ val locality = event.taskInfo.taskLocality.toString()
+ val count = stage.localitySummary.getOrElse(locality, 0L) + 1L
+ stage.localitySummary = stage.localitySummary ++ Map(locality -> count)
maybeUpdate(stage, now)
stage.jobs.foreach { job =>
@@ -433,7 +477,7 @@ private[spark] class AppStatusListener(
}
task.errorMessage = errorMessage
val delta = task.updateMetrics(event.taskMetrics)
- update(task, now)
+ update(task, now, last = true)
delta
}.orNull
@@ -450,7 +494,7 @@ private[spark] class AppStatusListener(
Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
if (metricsDelta != null) {
- stage.metrics.update(metricsDelta)
+ stage.metrics = LiveEntityHelpers.addMetrics(stage.metrics, metricsDelta)
}
stage.activeTasks -= 1
stage.completedTasks += completedDelta
@@ -486,7 +530,7 @@ private[spark] class AppStatusListener(
esummary.failedTasks += failedDelta
esummary.killedTasks += killedDelta
if (metricsDelta != null) {
- esummary.metrics.update(metricsDelta)
+ esummary.metrics = LiveEntityHelpers.addMetrics(esummary.metrics, metricsDelta)
}
maybeUpdate(esummary, now)
@@ -529,7 +573,8 @@ private[spark] class AppStatusListener(
}
override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
- val maybeStage = Option(liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptId)))
+ val maybeStage =
+ Option(liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber)))
maybeStage.foreach { stage =>
val now = System.nanoTime()
stage.info = event.stageInfo
@@ -564,12 +609,24 @@ private[spark] class AppStatusListener(
stage.executorSummaries.values.foreach(update(_, now))
update(stage, now, last = true)
+
+ val executorIdsForStage = stage.blackListedExecutors
+ executorIdsForStage.foreach { executorId =>
+ liveExecutors.get(executorId).foreach { exec =>
+ removeBlackListedStageFrom(exec, event.stageInfo.stageId, now)
+ }
+ }
}
appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1)
kvstore.write(appSummary)
}
+ private def removeBlackListedStageFrom(exec: LiveExecutor, stageId: Int, now: Long) = {
+ exec.blacklistedInStages -= stageId
+ liveUpdate(exec, now)
+ }
+
override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = {
// This needs to set fields that are already set by onExecutorAdded because the driver is
// considered an "executor" in the UI, but does not have a SparkListenerExecutorAdded event.
@@ -603,11 +660,11 @@ private[spark] class AppStatusListener(
maybeUpdate(task, now)
Option(liveStages.get((sid, sAttempt))).foreach { stage =>
- stage.metrics.update(delta)
+ stage.metrics = LiveEntityHelpers.addMetrics(stage.metrics, delta)
maybeUpdate(stage, now)
val esummary = stage.executorSummary(event.execId)
- esummary.metrics.update(delta)
+ esummary.metrics = LiveEntityHelpers.addMetrics(esummary.metrics, delta)
maybeUpdate(esummary, now)
}
}
@@ -689,7 +746,7 @@ private[spark] class AppStatusListener(
// can update the executor information too.
liveRDDs.get(block.rddId).foreach { rdd =>
if (updatedStorageLevel.isDefined) {
- rdd.storageLevel = updatedStorageLevel.get
+ rdd.setStorageLevel(updatedStorageLevel.get)
}
val partition = rdd.partition(block.name)
@@ -785,7 +842,7 @@ private[spark] class AppStatusListener(
}
private def getOrCreateStage(info: StageInfo): LiveStage = {
- val stage = liveStages.computeIfAbsent((info.stageId, info.attemptId),
+ val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber),
new Function[(Int, Int), LiveStage]() {
override def apply(key: (Int, Int)): LiveStage = new LiveStage()
})
@@ -813,7 +870,7 @@ private[spark] class AppStatusListener(
/** Update a live entity only if it hasn't been updated in the last configured period. */
private def maybeUpdate(entity: LiveEntity, now: Long): Unit = {
- if (liveUpdatePeriodNs >= 0 && now - entity.lastWriteTime > liveUpdatePeriodNs) {
+ if (live && liveUpdatePeriodNs >= 0 && now - entity.lastWriteTime > liveUpdatePeriodNs) {
update(entity, now)
}
}
@@ -845,8 +902,8 @@ private[spark] class AppStatusListener(
return
}
- val toDelete = KVUtils.viewToSeq(kvstore.view(classOf[JobDataWrapper]),
- countToDelete.toInt) { j =>
+ val view = kvstore.view(classOf[JobDataWrapper]).index("completionTime").first(0L)
+ val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j =>
j.info.status != JobExecutionStatus.RUNNING && j.info.status != JobExecutionStatus.UNKNOWN
}
toDelete.foreach { j => kvstore.delete(j.getClass(), j.info.jobId) }
@@ -858,13 +915,16 @@ private[spark] class AppStatusListener(
return
}
- val stages = KVUtils.viewToSeq(kvstore.view(classOf[StageDataWrapper]),
- countToDelete.toInt) { s =>
+ // As the completion time of a skipped stage is always -1, we will remove skipped stages first.
+ // This is safe since the job itself contains enough information to render skipped stages in the
+ // UI.
+ val view = kvstore.view(classOf[StageDataWrapper]).index("completionTime")
+ val stages = KVUtils.viewToSeq(view, countToDelete.toInt) { s =>
s.info.status != v1.StageStatus.ACTIVE && s.info.status != v1.StageStatus.PENDING
}
stages.foreach { s =>
- val key = s.id
+ val key = Array(s.info.stageId, s.info.attemptId)
kvstore.delete(s.getClass(), key)
val execSummaries = kvstore.view(classOf[ExecutorStageSummaryWrapper])
@@ -884,15 +944,15 @@ private[spark] class AppStatusListener(
.asScala
tasks.foreach { t =>
- kvstore.delete(t.getClass(), t.info.taskId)
+ kvstore.delete(t.getClass(), t.taskId)
}
// Check whether there are remaining attempts for the same stage. If there aren't, then
// also delete the RDD graph data.
val remainingAttempts = kvstore.view(classOf[StageDataWrapper])
.index("stageId")
- .first(s.stageId)
- .last(s.stageId)
+ .first(s.info.stageId)
+ .last(s.info.stageId)
.closeableIterator()
val hasMoreAttempts = try {
@@ -904,23 +964,26 @@ private[spark] class AppStatusListener(
}
if (!hasMoreAttempts) {
- kvstore.delete(classOf[RDDOperationGraphWrapper], s.stageId)
+ kvstore.delete(classOf[RDDOperationGraphWrapper], s.info.stageId)
}
+
+ cleanupCachedQuantiles(key)
}
}
private def cleanupTasks(stage: LiveStage): Unit = {
val countToDelete = calculateNumberToRemove(stage.savedTasks.get(), maxTasksPerStage).toInt
if (countToDelete > 0) {
- val stageKey = Array(stage.info.stageId, stage.info.attemptId)
- val view = kvstore.view(classOf[TaskDataWrapper]).index("stage").first(stageKey)
- .last(stageKey)
+ val stageKey = Array(stage.info.stageId, stage.info.attemptNumber)
+ val view = kvstore.view(classOf[TaskDataWrapper])
+ .index(TaskIndexNames.COMPLETION_TIME)
+ .parent(stageKey)
// Try to delete finished tasks only.
val toDelete = KVUtils.viewToSeq(view, countToDelete) { t =>
- !live || t.info.status != TaskState.RUNNING.toString()
+ !live || t.status != TaskState.RUNNING.toString()
}
- toDelete.foreach { t => kvstore.delete(t.getClass(), t.info.taskId) }
+ toDelete.foreach { t => kvstore.delete(t.getClass(), t.taskId) }
stage.savedTasks.addAndGet(-toDelete.size)
// If there are more running tasks than the configured limit, delete running tasks. This
@@ -929,13 +992,34 @@ private[spark] class AppStatusListener(
val remaining = countToDelete - toDelete.size
if (remaining > 0) {
val runningTasksToDelete = view.max(remaining).iterator().asScala.toList
- runningTasksToDelete.foreach { t => kvstore.delete(t.getClass(), t.info.taskId) }
+ runningTasksToDelete.foreach { t => kvstore.delete(t.getClass(), t.taskId) }
stage.savedTasks.addAndGet(-remaining)
}
+
+ // On live applications, cleanup any cached quantiles for the stage. This makes sure that
+ // quantiles will be recalculated after tasks are replaced with newer ones.
+ //
+ // This is not needed in the SHS since caching only happens after the event logs are
+ // completely processed.
+ if (live) {
+ cleanupCachedQuantiles(stageKey)
+ }
}
stage.cleaning = false
}
+ private def cleanupCachedQuantiles(stageKey: Array[Int]): Unit = {
+ val cachedQuantiles = kvstore.view(classOf[CachedQuantile])
+ .index("stage")
+ .first(stageKey)
+ .last(stageKey)
+ .asScala
+ .toList
+ cachedQuantiles.foreach { q =>
+ kvstore.delete(q.getClass(), q.id)
+ }
+ }
+
/**
* Remove at least (retainedSize / 10) items to reduce friction. Because tracking may be done
* asynchronously, this method may return 0 in case enough items have been deleted already.
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 5a942f528401..688f25a9fdea 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConverters._
import org.apache.spark.{JobExecutionStatus, SparkConf}
import org.apache.spark.status.api.v1
import org.apache.spark.ui.scope._
-import org.apache.spark.util.Distribution
+import org.apache.spark.util.{Distribution, Utils}
import org.apache.spark.util.kvstore.{InMemoryStore, KVStore}
/**
@@ -95,10 +95,18 @@ private[spark] class AppStatusStore(
}
def lastStageAttempt(stageId: Int): v1.StageData = {
- val it = store.view(classOf[StageDataWrapper]).index("stageId").reverse().first(stageId)
+ val it = store.view(classOf[StageDataWrapper])
+ .index("stageId")
+ .reverse()
+ .first(stageId)
+ .last(stageId)
.closeableIterator()
try {
- it.next().info
+ if (it.hasNext()) {
+ it.next().info
+ } else {
+ throw new NoSuchElementException(s"No stage with id $stageId")
+ }
} finally {
it.close()
}
@@ -110,107 +118,238 @@ private[spark] class AppStatusStore(
if (details) stageWithDetails(stage) else stage
}
+ def taskCount(stageId: Int, stageAttemptId: Int): Long = {
+ store.count(classOf[TaskDataWrapper], "stage", Array(stageId, stageAttemptId))
+ }
+
+ def localitySummary(stageId: Int, stageAttemptId: Int): Map[String, Long] = {
+ store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).locality
+ }
+
+ /**
+ * Calculates a summary of the task metrics for the given stage attempt, returning the
+ * requested quantiles for the recorded metrics.
+ *
+ * This method can be expensive if the requested quantiles are not cached; the method
+ * will only cache certain quantiles (every 0.05 step), so it's recommended to stick to
+ * those to avoid expensive scans of all task data.
+ */
def taskSummary(
stageId: Int,
stageAttemptId: Int,
- quantiles: Array[Double]): v1.TaskMetricDistributions = {
-
- val stage = Array(stageId, stageAttemptId)
-
- val rawMetrics = store.view(classOf[TaskDataWrapper])
- .index("stage")
- .first(stage)
- .last(stage)
- .asScala
- .flatMap(_.info.taskMetrics)
- .toList
- .view
-
- def metricQuantiles(f: v1.TaskMetrics => Double): IndexedSeq[Double] =
- Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles)
-
- // We need to do a lot of similar munging to nested metrics here. For each one,
- // we want (a) extract the values for nested metrics (b) make a distribution for each metric
- // (c) shove the distribution into the right field in our return type and (d) only return
- // a result if the option is defined for any of the tasks. MetricHelper is a little util
- // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just
- // implement one "build" method, which just builds the quantiles for each field.
-
- val inputMetrics =
- new MetricHelper[v1.InputMetrics, v1.InputMetricDistributions](rawMetrics, quantiles) {
- def getSubmetrics(raw: v1.TaskMetrics): v1.InputMetrics = raw.inputMetrics
-
- def build: v1.InputMetricDistributions = new v1.InputMetricDistributions(
- bytesRead = submetricQuantiles(_.bytesRead),
- recordsRead = submetricQuantiles(_.recordsRead)
- )
- }.build
-
- val outputMetrics =
- new MetricHelper[v1.OutputMetrics, v1.OutputMetricDistributions](rawMetrics, quantiles) {
- def getSubmetrics(raw: v1.TaskMetrics): v1.OutputMetrics = raw.outputMetrics
-
- def build: v1.OutputMetricDistributions = new v1.OutputMetricDistributions(
- bytesWritten = submetricQuantiles(_.bytesWritten),
- recordsWritten = submetricQuantiles(_.recordsWritten)
- )
- }.build
-
- val shuffleReadMetrics =
- new MetricHelper[v1.ShuffleReadMetrics, v1.ShuffleReadMetricDistributions](rawMetrics,
- quantiles) {
- def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleReadMetrics =
- raw.shuffleReadMetrics
-
- def build: v1.ShuffleReadMetricDistributions = new v1.ShuffleReadMetricDistributions(
- readBytes = submetricQuantiles { s => s.localBytesRead + s.remoteBytesRead },
- readRecords = submetricQuantiles(_.recordsRead),
- remoteBytesRead = submetricQuantiles(_.remoteBytesRead),
- remoteBytesReadToDisk = submetricQuantiles(_.remoteBytesReadToDisk),
- remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched),
- localBlocksFetched = submetricQuantiles(_.localBlocksFetched),
- totalBlocksFetched = submetricQuantiles { s =>
- s.localBlocksFetched + s.remoteBlocksFetched
- },
- fetchWaitTime = submetricQuantiles(_.fetchWaitTime)
- )
- }.build
-
- val shuffleWriteMetrics =
- new MetricHelper[v1.ShuffleWriteMetrics, v1.ShuffleWriteMetricDistributions](rawMetrics,
- quantiles) {
- def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleWriteMetrics =
- raw.shuffleWriteMetrics
-
- def build: v1.ShuffleWriteMetricDistributions = new v1.ShuffleWriteMetricDistributions(
- writeBytes = submetricQuantiles(_.bytesWritten),
- writeRecords = submetricQuantiles(_.recordsWritten),
- writeTime = submetricQuantiles(_.writeTime)
- )
- }.build
-
- new v1.TaskMetricDistributions(
+ unsortedQuantiles: Array[Double]): Option[v1.TaskMetricDistributions] = {
+ val stageKey = Array(stageId, stageAttemptId)
+ val quantiles = unsortedQuantiles.sorted
+
+ // We don't know how many tasks remain in the store that actually have metrics. So scan one
+ // metric and count how many valid tasks there are. Use skip() instead of next() since it's
+ // cheaper for disk stores (avoids deserialization).
+ val count = {
+ Utils.tryWithResource(
+ store.view(classOf[TaskDataWrapper])
+ .parent(stageKey)
+ .index(TaskIndexNames.EXEC_RUN_TIME)
+ .first(0L)
+ .closeableIterator()
+ ) { it =>
+ var _count = 0L
+ while (it.hasNext()) {
+ _count += 1
+ it.skip(1)
+ }
+ _count
+ }
+ }
+
+ if (count <= 0) {
+ return None
+ }
+
+ // Find out which quantiles are already cached. The data in the store must match the expected
+ // task count to be considered, otherwise it will be re-scanned and overwritten.
+ val cachedQuantiles = quantiles.filter(shouldCacheQuantile).flatMap { q =>
+ val qkey = Array(stageId, stageAttemptId, quantileToString(q))
+ asOption(store.read(classOf[CachedQuantile], qkey)).filter(_.taskCount == count)
+ }
+
+ // If there are no missing quantiles, return the data. Otherwise, just compute everything
+ // to make the code simpler.
+ if (cachedQuantiles.size == quantiles.size) {
+ def toValues(fn: CachedQuantile => Double): IndexedSeq[Double] = cachedQuantiles.map(fn)
+
+ val distributions = new v1.TaskMetricDistributions(
+ quantiles = quantiles,
+ executorDeserializeTime = toValues(_.executorDeserializeTime),
+ executorDeserializeCpuTime = toValues(_.executorDeserializeCpuTime),
+ executorRunTime = toValues(_.executorRunTime),
+ executorCpuTime = toValues(_.executorCpuTime),
+ resultSize = toValues(_.resultSize),
+ jvmGcTime = toValues(_.jvmGcTime),
+ resultSerializationTime = toValues(_.resultSerializationTime),
+ gettingResultTime = toValues(_.gettingResultTime),
+ schedulerDelay = toValues(_.schedulerDelay),
+ peakExecutionMemory = toValues(_.peakExecutionMemory),
+ memoryBytesSpilled = toValues(_.memoryBytesSpilled),
+ diskBytesSpilled = toValues(_.diskBytesSpilled),
+ inputMetrics = new v1.InputMetricDistributions(
+ toValues(_.bytesRead),
+ toValues(_.recordsRead)),
+ outputMetrics = new v1.OutputMetricDistributions(
+ toValues(_.bytesWritten),
+ toValues(_.recordsWritten)),
+ shuffleReadMetrics = new v1.ShuffleReadMetricDistributions(
+ toValues(_.shuffleReadBytes),
+ toValues(_.shuffleRecordsRead),
+ toValues(_.shuffleRemoteBlocksFetched),
+ toValues(_.shuffleLocalBlocksFetched),
+ toValues(_.shuffleFetchWaitTime),
+ toValues(_.shuffleRemoteBytesRead),
+ toValues(_.shuffleRemoteBytesReadToDisk),
+ toValues(_.shuffleTotalBlocksFetched)),
+ shuffleWriteMetrics = new v1.ShuffleWriteMetricDistributions(
+ toValues(_.shuffleWriteBytes),
+ toValues(_.shuffleWriteRecords),
+ toValues(_.shuffleWriteTime)))
+
+ return Some(distributions)
+ }
+
+ // Compute quantiles by scanning the tasks in the store. This is not really stable for live
+ // stages (e.g. the number of recorded tasks may change while this code is running), but should
+ // stabilize once the stage finishes. It's also slow, especially with disk stores.
+ val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) }
+
+ def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = {
+ Utils.tryWithResource(
+ store.view(classOf[TaskDataWrapper])
+ .parent(stageKey)
+ .index(index)
+ .first(0L)
+ .closeableIterator()
+ ) { it =>
+ var last = Double.NaN
+ var currentIdx = -1L
+ indices.map { idx =>
+ if (idx == currentIdx) {
+ last
+ } else {
+ val diff = idx - currentIdx
+ currentIdx = idx
+ if (it.skip(diff - 1)) {
+ last = fn(it.next()).toDouble
+ last
+ } else {
+ Double.NaN
+ }
+ }
+ }.toIndexedSeq
+ }
+ }
+
+ val computedQuantiles = new v1.TaskMetricDistributions(
quantiles = quantiles,
- executorDeserializeTime = metricQuantiles(_.executorDeserializeTime),
- executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime),
- executorRunTime = metricQuantiles(_.executorRunTime),
- executorCpuTime = metricQuantiles(_.executorCpuTime),
- resultSize = metricQuantiles(_.resultSize),
- jvmGcTime = metricQuantiles(_.jvmGcTime),
- resultSerializationTime = metricQuantiles(_.resultSerializationTime),
- memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled),
- diskBytesSpilled = metricQuantiles(_.diskBytesSpilled),
- inputMetrics = inputMetrics,
- outputMetrics = outputMetrics,
- shuffleReadMetrics = shuffleReadMetrics,
- shuffleWriteMetrics = shuffleWriteMetrics
- )
+ executorDeserializeTime = scanTasks(TaskIndexNames.DESER_TIME) { t =>
+ t.executorDeserializeTime
+ },
+ executorDeserializeCpuTime = scanTasks(TaskIndexNames.DESER_CPU_TIME) { t =>
+ t.executorDeserializeCpuTime
+ },
+ executorRunTime = scanTasks(TaskIndexNames.EXEC_RUN_TIME) { t => t.executorRunTime },
+ executorCpuTime = scanTasks(TaskIndexNames.EXEC_CPU_TIME) { t => t.executorCpuTime },
+ resultSize = scanTasks(TaskIndexNames.RESULT_SIZE) { t => t.resultSize },
+ jvmGcTime = scanTasks(TaskIndexNames.GC_TIME) { t => t.jvmGcTime },
+ resultSerializationTime = scanTasks(TaskIndexNames.SER_TIME) { t =>
+ t.resultSerializationTime
+ },
+ gettingResultTime = scanTasks(TaskIndexNames.GETTING_RESULT_TIME) { t =>
+ t.gettingResultTime
+ },
+ schedulerDelay = scanTasks(TaskIndexNames.SCHEDULER_DELAY) { t => t.schedulerDelay },
+ peakExecutionMemory = scanTasks(TaskIndexNames.PEAK_MEM) { t => t.peakExecutionMemory },
+ memoryBytesSpilled = scanTasks(TaskIndexNames.MEM_SPILL) { t => t.memoryBytesSpilled },
+ diskBytesSpilled = scanTasks(TaskIndexNames.DISK_SPILL) { t => t.diskBytesSpilled },
+ inputMetrics = new v1.InputMetricDistributions(
+ scanTasks(TaskIndexNames.INPUT_SIZE) { t => t.inputBytesRead },
+ scanTasks(TaskIndexNames.INPUT_RECORDS) { t => t.inputRecordsRead }),
+ outputMetrics = new v1.OutputMetricDistributions(
+ scanTasks(TaskIndexNames.OUTPUT_SIZE) { t => t.outputBytesWritten },
+ scanTasks(TaskIndexNames.OUTPUT_RECORDS) { t => t.outputRecordsWritten }),
+ shuffleReadMetrics = new v1.ShuffleReadMetricDistributions(
+ scanTasks(TaskIndexNames.SHUFFLE_TOTAL_READS) { m =>
+ m.shuffleLocalBytesRead + m.shuffleRemoteBytesRead
+ },
+ scanTasks(TaskIndexNames.SHUFFLE_READ_RECORDS) { t => t.shuffleRecordsRead },
+ scanTasks(TaskIndexNames.SHUFFLE_REMOTE_BLOCKS) { t => t.shuffleRemoteBlocksFetched },
+ scanTasks(TaskIndexNames.SHUFFLE_LOCAL_BLOCKS) { t => t.shuffleLocalBlocksFetched },
+ scanTasks(TaskIndexNames.SHUFFLE_READ_TIME) { t => t.shuffleFetchWaitTime },
+ scanTasks(TaskIndexNames.SHUFFLE_REMOTE_READS) { t => t.shuffleRemoteBytesRead },
+ scanTasks(TaskIndexNames.SHUFFLE_REMOTE_READS_TO_DISK) { t =>
+ t.shuffleRemoteBytesReadToDisk
+ },
+ scanTasks(TaskIndexNames.SHUFFLE_TOTAL_BLOCKS) { m =>
+ m.shuffleLocalBlocksFetched + m.shuffleRemoteBlocksFetched
+ }),
+ shuffleWriteMetrics = new v1.ShuffleWriteMetricDistributions(
+ scanTasks(TaskIndexNames.SHUFFLE_WRITE_SIZE) { t => t.shuffleBytesWritten },
+ scanTasks(TaskIndexNames.SHUFFLE_WRITE_RECORDS) { t => t.shuffleRecordsWritten },
+ scanTasks(TaskIndexNames.SHUFFLE_WRITE_TIME) { t => t.shuffleWriteTime }))
+
+ // Go through the computed quantiles and cache the values that match the caching criteria.
+ computedQuantiles.quantiles.zipWithIndex
+ .filter { case (q, _) => quantiles.contains(q) && shouldCacheQuantile(q) }
+ .foreach { case (q, idx) =>
+ val cached = new CachedQuantile(stageId, stageAttemptId, quantileToString(q), count,
+ executorDeserializeTime = computedQuantiles.executorDeserializeTime(idx),
+ executorDeserializeCpuTime = computedQuantiles.executorDeserializeCpuTime(idx),
+ executorRunTime = computedQuantiles.executorRunTime(idx),
+ executorCpuTime = computedQuantiles.executorCpuTime(idx),
+ resultSize = computedQuantiles.resultSize(idx),
+ jvmGcTime = computedQuantiles.jvmGcTime(idx),
+ resultSerializationTime = computedQuantiles.resultSerializationTime(idx),
+ gettingResultTime = computedQuantiles.gettingResultTime(idx),
+ schedulerDelay = computedQuantiles.schedulerDelay(idx),
+ peakExecutionMemory = computedQuantiles.peakExecutionMemory(idx),
+ memoryBytesSpilled = computedQuantiles.memoryBytesSpilled(idx),
+ diskBytesSpilled = computedQuantiles.diskBytesSpilled(idx),
+
+ bytesRead = computedQuantiles.inputMetrics.bytesRead(idx),
+ recordsRead = computedQuantiles.inputMetrics.recordsRead(idx),
+
+ bytesWritten = computedQuantiles.outputMetrics.bytesWritten(idx),
+ recordsWritten = computedQuantiles.outputMetrics.recordsWritten(idx),
+
+ shuffleReadBytes = computedQuantiles.shuffleReadMetrics.readBytes(idx),
+ shuffleRecordsRead = computedQuantiles.shuffleReadMetrics.readRecords(idx),
+ shuffleRemoteBlocksFetched =
+ computedQuantiles.shuffleReadMetrics.remoteBlocksFetched(idx),
+ shuffleLocalBlocksFetched = computedQuantiles.shuffleReadMetrics.localBlocksFetched(idx),
+ shuffleFetchWaitTime = computedQuantiles.shuffleReadMetrics.fetchWaitTime(idx),
+ shuffleRemoteBytesRead = computedQuantiles.shuffleReadMetrics.remoteBytesRead(idx),
+ shuffleRemoteBytesReadToDisk =
+ computedQuantiles.shuffleReadMetrics.remoteBytesReadToDisk(idx),
+ shuffleTotalBlocksFetched = computedQuantiles.shuffleReadMetrics.totalBlocksFetched(idx),
+
+ shuffleWriteBytes = computedQuantiles.shuffleWriteMetrics.writeBytes(idx),
+ shuffleWriteRecords = computedQuantiles.shuffleWriteMetrics.writeRecords(idx),
+ shuffleWriteTime = computedQuantiles.shuffleWriteMetrics.writeTime(idx))
+ store.write(cached)
+ }
+
+ Some(computedQuantiles)
}
+ /**
+ * Whether to cache information about a specific metric quantile. We cache quantiles at every 0.05
+ * step, which covers the default values used both in the API and in the stages page.
+ */
+ private def shouldCacheQuantile(q: Double): Boolean = (math.round(q * 100) % 5) == 0
+
+ private def quantileToString(q: Double): String = math.round(q * 100).toString
+
def taskList(stageId: Int, stageAttemptId: Int, maxTasks: Int): Seq[v1.TaskData] = {
val stageKey = Array(stageId, stageAttemptId)
store.view(classOf[TaskDataWrapper]).index("stage").first(stageKey).last(stageKey).reverse()
- .max(maxTasks).asScala.map(_.info).toSeq.reverse
+ .max(maxTasks).asScala.map(_.toApi).toSeq.reverse
}
def taskList(
@@ -219,18 +358,43 @@ private[spark] class AppStatusStore(
offset: Int,
length: Int,
sortBy: v1.TaskSorting): Seq[v1.TaskData] = {
+ val (indexName, ascending) = sortBy match {
+ case v1.TaskSorting.ID =>
+ (None, true)
+ case v1.TaskSorting.INCREASING_RUNTIME =>
+ (Some(TaskIndexNames.EXEC_RUN_TIME), true)
+ case v1.TaskSorting.DECREASING_RUNTIME =>
+ (Some(TaskIndexNames.EXEC_RUN_TIME), false)
+ }
+ taskList(stageId, stageAttemptId, offset, length, indexName, ascending)
+ }
+
+ def taskList(
+ stageId: Int,
+ stageAttemptId: Int,
+ offset: Int,
+ length: Int,
+ sortBy: Option[String],
+ ascending: Boolean): Seq[v1.TaskData] = {
val stageKey = Array(stageId, stageAttemptId)
val base = store.view(classOf[TaskDataWrapper])
val indexed = sortBy match {
- case v1.TaskSorting.ID =>
+ case Some(index) =>
+ base.index(index).parent(stageKey)
+
+ case _ =>
+ // Sort by ID, which is the "stage" index.
base.index("stage").first(stageKey).last(stageKey)
- case v1.TaskSorting.INCREASING_RUNTIME =>
- base.index("runtime").first(stageKey ++ Array(-1L)).last(stageKey ++ Array(Long.MaxValue))
- case v1.TaskSorting.DECREASING_RUNTIME =>
- base.index("runtime").first(stageKey ++ Array(Long.MaxValue)).last(stageKey ++ Array(-1L))
- .reverse()
}
- indexed.skip(offset).max(length).asScala.map(_.info).toSeq
+
+ val ordered = if (ascending) indexed else indexed.reverse()
+ ordered.skip(offset).max(length).asScala.map(_.toApi).toSeq
+ }
+
+ def executorSummary(stageId: Int, attemptId: Int): Map[String, v1.ExecutorStageSummary] = {
+ val stageKey = Array(stageId, attemptId)
+ store.view(classOf[ExecutorStageSummaryWrapper]).index("stage").first(stageKey).last(stageKey)
+ .asScala.map { exec => (exec.executorId -> exec.info) }.toMap
}
def rddList(cachedOnly: Boolean = true): Seq[v1.RDDStorageInfo] = {
@@ -256,12 +420,6 @@ private[spark] class AppStatusStore(
.map { t => (t.taskId, t) }
.toMap
- val stageKey = Array(stage.stageId, stage.attemptId)
- val execs = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage").first(stageKey)
- .last(stageKey).closeableIterator().asScala
- .map { exec => (exec.executorId -> exec.info) }
- .toMap
-
new v1.StageData(
stage.status,
stage.stageId,
@@ -295,7 +453,7 @@ private[spark] class AppStatusStore(
stage.rddIds,
stage.accumulatorUpdates,
Some(tasks),
- Some(execs),
+ Some(executorSummary(stage.stageId, stage.attemptId)),
stage.killedTasksSummary)
}
@@ -352,22 +510,3 @@ private[spark] object AppStatusStore {
}
}
-
-/**
- * Helper for getting distributions from nested metric types.
- */
-private abstract class MetricHelper[I, O](
- rawMetrics: Seq[v1.TaskMetrics],
- quantiles: Array[Double]) {
-
- def getSubmetrics(raw: v1.TaskMetrics): I
-
- def build: O
-
- val data: Seq[I] = rawMetrics.map(getSubmetrics)
-
- /** applies the given function to all input metrics, and returns the quantiles */
- def submetricQuantiles(f: I => Double): IndexedSeq[Double] = {
- Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles)
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusUtils.scala b/core/src/main/scala/org/apache/spark/status/AppStatusUtils.scala
new file mode 100644
index 000000000000..87f434daf487
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusUtils.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.status
+
+import org.apache.spark.status.api.v1.TaskData
+
+private[spark] object AppStatusUtils {
+
+ private val TASK_FINISHED_STATES = Set("FAILED", "KILLED", "SUCCESS")
+
+ private def isTaskFinished(task: TaskData): Boolean = {
+ TASK_FINISHED_STATES.contains(task.status)
+ }
+
+ def schedulerDelay(task: TaskData): Long = {
+ if (isTaskFinished(task) && task.taskMetrics.isDefined && task.duration.isDefined) {
+ val m = task.taskMetrics.get
+ schedulerDelay(task.launchTime.getTime(), fetchStart(task), task.duration.get,
+ m.executorDeserializeTime, m.resultSerializationTime, m.executorRunTime)
+ } else {
+ // The task is still running and the metrics like executorRunTime are not available.
+ 0L
+ }
+ }
+
+ def gettingResultTime(task: TaskData): Long = {
+ gettingResultTime(task.launchTime.getTime(), fetchStart(task), task.duration.getOrElse(-1L))
+ }
+
+ def schedulerDelay(
+ launchTime: Long,
+ fetchStart: Long,
+ duration: Long,
+ deserializeTime: Long,
+ serializeTime: Long,
+ runTime: Long): Long = {
+ math.max(0, duration - runTime - deserializeTime - serializeTime -
+ gettingResultTime(launchTime, fetchStart, duration))
+ }
+
+ def gettingResultTime(launchTime: Long, fetchStart: Long, duration: Long): Long = {
+ if (fetchStart > 0) {
+ if (duration > 0) {
+ launchTime + duration - fetchStart
+ } else {
+ System.currentTimeMillis() - fetchStart
+ }
+ } else {
+ 0L
+ }
+ }
+
+ private def fetchStart(task: TaskData): Long = {
+ if (task.resultFetchStart.isDefined) {
+ task.resultFetchStart.get.getTime()
+ } else {
+ -1
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index 52e83f250d34..79e3f13b826c 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -20,8 +20,11 @@ package org.apache.spark.status
import java.util.Date
import java.util.concurrent.atomic.AtomicInteger
+import scala.collection.immutable.{HashSet, TreeSet}
import scala.collection.mutable.HashMap
+import com.google.common.collect.Interners
+
import org.apache.spark.JobExecutionStatus
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo}
@@ -119,7 +122,9 @@ private class LiveTask(
import LiveEntityHelpers._
- private var recordedMetrics: v1.TaskMetrics = null
+ // The task metrics use a special value when no metrics have been reported. The special value is
+ // checked when calculating indexed values when writing to the store (see [[TaskDataWrapper]]).
+ private var metrics: v1.TaskMetrics = createMetrics(default = -1L)
var errorMessage: Option[String] = None
@@ -129,8 +134,8 @@ private class LiveTask(
*/
def updateMetrics(metrics: TaskMetrics): v1.TaskMetrics = {
if (metrics != null) {
- val old = recordedMetrics
- recordedMetrics = new v1.TaskMetrics(
+ val old = this.metrics
+ val newMetrics = createMetrics(
metrics.executorDeserializeTime,
metrics.executorDeserializeCpuTime,
metrics.executorRunTime,
@@ -141,73 +146,35 @@ private class LiveTask(
metrics.memoryBytesSpilled,
metrics.diskBytesSpilled,
metrics.peakExecutionMemory,
- new v1.InputMetrics(
- metrics.inputMetrics.bytesRead,
- metrics.inputMetrics.recordsRead),
- new v1.OutputMetrics(
- metrics.outputMetrics.bytesWritten,
- metrics.outputMetrics.recordsWritten),
- new v1.ShuffleReadMetrics(
- metrics.shuffleReadMetrics.remoteBlocksFetched,
- metrics.shuffleReadMetrics.localBlocksFetched,
- metrics.shuffleReadMetrics.fetchWaitTime,
- metrics.shuffleReadMetrics.remoteBytesRead,
- metrics.shuffleReadMetrics.remoteBytesReadToDisk,
- metrics.shuffleReadMetrics.localBytesRead,
- metrics.shuffleReadMetrics.recordsRead),
- new v1.ShuffleWriteMetrics(
- metrics.shuffleWriteMetrics.bytesWritten,
- metrics.shuffleWriteMetrics.writeTime,
- metrics.shuffleWriteMetrics.recordsWritten))
- if (old != null) calculateMetricsDelta(recordedMetrics, old) else recordedMetrics
+ metrics.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead,
+ metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten,
+ metrics.shuffleReadMetrics.remoteBlocksFetched,
+ metrics.shuffleReadMetrics.localBlocksFetched,
+ metrics.shuffleReadMetrics.fetchWaitTime,
+ metrics.shuffleReadMetrics.remoteBytesRead,
+ metrics.shuffleReadMetrics.remoteBytesReadToDisk,
+ metrics.shuffleReadMetrics.localBytesRead,
+ metrics.shuffleReadMetrics.recordsRead,
+ metrics.shuffleWriteMetrics.bytesWritten,
+ metrics.shuffleWriteMetrics.writeTime,
+ metrics.shuffleWriteMetrics.recordsWritten)
+
+ this.metrics = newMetrics
+
+ // Only calculate the delta if the old metrics contain valid information, otherwise
+ // the new metrics are the delta.
+ if (old.executorDeserializeTime >= 0L) {
+ subtractMetrics(newMetrics, old)
+ } else {
+ newMetrics
+ }
} else {
null
}
}
- /**
- * Return a new TaskMetrics object containing the delta of the various fields of the given
- * metrics objects. This is currently targeted at updating stage data, so it does not
- * necessarily calculate deltas for all the fields.
- */
- private def calculateMetricsDelta(
- metrics: v1.TaskMetrics,
- old: v1.TaskMetrics): v1.TaskMetrics = {
- val shuffleWriteDelta = new v1.ShuffleWriteMetrics(
- metrics.shuffleWriteMetrics.bytesWritten - old.shuffleWriteMetrics.bytesWritten,
- 0L,
- metrics.shuffleWriteMetrics.recordsWritten - old.shuffleWriteMetrics.recordsWritten)
-
- val shuffleReadDelta = new v1.ShuffleReadMetrics(
- 0L, 0L, 0L,
- metrics.shuffleReadMetrics.remoteBytesRead - old.shuffleReadMetrics.remoteBytesRead,
- metrics.shuffleReadMetrics.remoteBytesReadToDisk -
- old.shuffleReadMetrics.remoteBytesReadToDisk,
- metrics.shuffleReadMetrics.localBytesRead - old.shuffleReadMetrics.localBytesRead,
- metrics.shuffleReadMetrics.recordsRead - old.shuffleReadMetrics.recordsRead)
-
- val inputDelta = new v1.InputMetrics(
- metrics.inputMetrics.bytesRead - old.inputMetrics.bytesRead,
- metrics.inputMetrics.recordsRead - old.inputMetrics.recordsRead)
-
- val outputDelta = new v1.OutputMetrics(
- metrics.outputMetrics.bytesWritten - old.outputMetrics.bytesWritten,
- metrics.outputMetrics.recordsWritten - old.outputMetrics.recordsWritten)
-
- new v1.TaskMetrics(
- 0L, 0L,
- metrics.executorRunTime - old.executorRunTime,
- metrics.executorCpuTime - old.executorCpuTime,
- 0L, 0L, 0L,
- metrics.memoryBytesSpilled - old.memoryBytesSpilled,
- metrics.diskBytesSpilled - old.diskBytesSpilled,
- 0L,
- inputDelta,
- outputDelta,
- shuffleReadDelta,
- shuffleWriteDelta)
- }
-
override protected def doUpdate(): Any = {
val duration = if (info.finished) {
info.duration
@@ -215,22 +182,48 @@ private class LiveTask(
info.timeRunning(lastUpdateTime.getOrElse(System.currentTimeMillis()))
}
- val task = new v1.TaskData(
+ new TaskDataWrapper(
info.taskId,
info.index,
info.attemptNumber,
- new Date(info.launchTime),
- if (info.gettingResult) Some(new Date(info.gettingResultTime)) else None,
- Some(duration),
- info.executorId,
- info.host,
- info.status,
- info.taskLocality.toString(),
+ info.launchTime,
+ if (info.gettingResult) info.gettingResultTime else -1L,
+ duration,
+ weakIntern(info.executorId),
+ weakIntern(info.host),
+ weakIntern(info.status),
+ weakIntern(info.taskLocality.toString()),
info.speculative,
newAccumulatorInfos(info.accumulables),
errorMessage,
- Option(recordedMetrics))
- new TaskDataWrapper(task, stageId, stageAttemptId)
+
+ metrics.executorDeserializeTime,
+ metrics.executorDeserializeCpuTime,
+ metrics.executorRunTime,
+ metrics.executorCpuTime,
+ metrics.resultSize,
+ metrics.jvmGcTime,
+ metrics.resultSerializationTime,
+ metrics.memoryBytesSpilled,
+ metrics.diskBytesSpilled,
+ metrics.peakExecutionMemory,
+ metrics.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead,
+ metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten,
+ metrics.shuffleReadMetrics.remoteBlocksFetched,
+ metrics.shuffleReadMetrics.localBlocksFetched,
+ metrics.shuffleReadMetrics.fetchWaitTime,
+ metrics.shuffleReadMetrics.remoteBytesRead,
+ metrics.shuffleReadMetrics.remoteBytesReadToDisk,
+ metrics.shuffleReadMetrics.localBytesRead,
+ metrics.shuffleReadMetrics.recordsRead,
+ metrics.shuffleWriteMetrics.bytesWritten,
+ metrics.shuffleWriteMetrics.writeTime,
+ metrics.shuffleWriteMetrics.recordsWritten,
+
+ stageId,
+ stageAttemptId)
}
}
@@ -262,6 +255,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
var totalShuffleRead = 0L
var totalShuffleWrite = 0L
var isBlacklisted = false
+ var blacklistedInStages: Set[Int] = TreeSet()
var executorLogs = Map[String, String]()
@@ -307,56 +301,27 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
Option(removeTime),
Option(removeReason),
executorLogs,
- memoryMetrics)
+ memoryMetrics,
+ blacklistedInStages)
new ExecutorSummaryWrapper(info)
}
}
-/** Metrics tracked per stage (both total and per executor). */
-private class MetricsTracker {
- var executorRunTime = 0L
- var executorCpuTime = 0L
- var inputBytes = 0L
- var inputRecords = 0L
- var outputBytes = 0L
- var outputRecords = 0L
- var shuffleReadBytes = 0L
- var shuffleReadRecords = 0L
- var shuffleWriteBytes = 0L
- var shuffleWriteRecords = 0L
- var memoryBytesSpilled = 0L
- var diskBytesSpilled = 0L
-
- def update(delta: v1.TaskMetrics): Unit = {
- executorRunTime += delta.executorRunTime
- executorCpuTime += delta.executorCpuTime
- inputBytes += delta.inputMetrics.bytesRead
- inputRecords += delta.inputMetrics.recordsRead
- outputBytes += delta.outputMetrics.bytesWritten
- outputRecords += delta.outputMetrics.recordsWritten
- shuffleReadBytes += delta.shuffleReadMetrics.localBytesRead +
- delta.shuffleReadMetrics.remoteBytesRead
- shuffleReadRecords += delta.shuffleReadMetrics.recordsRead
- shuffleWriteBytes += delta.shuffleWriteMetrics.bytesWritten
- shuffleWriteRecords += delta.shuffleWriteMetrics.recordsWritten
- memoryBytesSpilled += delta.memoryBytesSpilled
- diskBytesSpilled += delta.diskBytesSpilled
- }
-
-}
-
private class LiveExecutorStageSummary(
stageId: Int,
attemptId: Int,
executorId: String) extends LiveEntity {
+ import LiveEntityHelpers._
+
var taskTime = 0L
var succeededTasks = 0
var failedTasks = 0
var killedTasks = 0
+ var isBlacklisted = false
- val metrics = new MetricsTracker()
+ var metrics = createMetrics(default = 0L)
override protected def doUpdate(): Any = {
val info = new v1.ExecutorStageSummary(
@@ -364,16 +329,17 @@ private class LiveExecutorStageSummary(
failedTasks,
succeededTasks,
killedTasks,
- metrics.inputBytes,
- metrics.inputRecords,
- metrics.outputBytes,
- metrics.outputRecords,
- metrics.shuffleReadBytes,
- metrics.shuffleReadRecords,
- metrics.shuffleWriteBytes,
- metrics.shuffleWriteRecords,
+ metrics.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead,
+ metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten,
+ metrics.shuffleReadMetrics.remoteBytesRead + metrics.shuffleReadMetrics.localBytesRead,
+ metrics.shuffleReadMetrics.recordsRead,
+ metrics.shuffleWriteMetrics.bytesWritten,
+ metrics.shuffleWriteMetrics.recordsWritten,
metrics.memoryBytesSpilled,
- metrics.diskBytesSpilled)
+ metrics.diskBytesSpilled,
+ isBlacklisted)
new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info)
}
@@ -402,24 +368,28 @@ private class LiveStage extends LiveEntity {
var firstLaunchTime = Long.MaxValue
- val metrics = new MetricsTracker()
+ var localitySummary: Map[String, Long] = Map()
+
+ var metrics = createMetrics(default = 0L)
val executorSummaries = new HashMap[String, LiveExecutorStageSummary]()
+ var blackListedExecutors = new HashSet[String]()
+
// Used for cleanup of tasks after they reach the configured limit. Not written to the store.
@volatile var cleaning = false
var savedTasks = new AtomicInteger(0)
def executorSummary(executorId: String): LiveExecutorStageSummary = {
executorSummaries.getOrElseUpdate(executorId,
- new LiveExecutorStageSummary(info.stageId, info.attemptId, executorId))
+ new LiveExecutorStageSummary(info.stageId, info.attemptNumber, executorId))
}
def toApi(): v1.StageData = {
new v1.StageData(
status,
info.stageId,
- info.attemptId,
+ info.attemptNumber,
info.numTasks,
activeTasks,
@@ -435,14 +405,14 @@ private class LiveStage extends LiveEntity {
info.completionTime.map(new Date(_)),
info.failureReason,
- metrics.inputBytes,
- metrics.inputRecords,
- metrics.outputBytes,
- metrics.outputRecords,
- metrics.shuffleReadBytes,
- metrics.shuffleReadRecords,
- metrics.shuffleWriteBytes,
- metrics.shuffleWriteRecords,
+ metrics.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead,
+ metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten,
+ metrics.shuffleReadMetrics.localBytesRead + metrics.shuffleReadMetrics.remoteBytesRead,
+ metrics.shuffleReadMetrics.recordsRead,
+ metrics.shuffleWriteMetrics.bytesWritten,
+ metrics.shuffleWriteMetrics.recordsWritten,
metrics.memoryBytesSpilled,
metrics.diskBytesSpilled,
@@ -459,13 +429,15 @@ private class LiveStage extends LiveEntity {
}
override protected def doUpdate(): Any = {
- new StageDataWrapper(toApi(), jobIds)
+ new StageDataWrapper(toApi(), jobIds, localitySummary)
}
}
private class LiveRDDPartition(val blockName: String) {
+ import LiveEntityHelpers._
+
// Pointers used by RDDPartitionSeq.
@volatile var prev: LiveRDDPartition = null
@volatile var next: LiveRDDPartition = null
@@ -485,7 +457,7 @@ private class LiveRDDPartition(val blockName: String) {
diskUsed: Long): Unit = {
value = new v1.RDDPartitionInfo(
blockName,
- storageLevel,
+ weakIntern(storageLevel),
memoryUsed,
diskUsed,
executors)
@@ -495,6 +467,8 @@ private class LiveRDDPartition(val blockName: String) {
private class LiveRDDDistribution(exec: LiveExecutor) {
+ import LiveEntityHelpers._
+
val executorId = exec.executorId
var memoryUsed = 0L
var diskUsed = 0L
@@ -508,7 +482,7 @@ private class LiveRDDDistribution(exec: LiveExecutor) {
def toApi(): v1.RDDDataDistribution = {
if (lastUpdate == null) {
lastUpdate = new v1.RDDDataDistribution(
- exec.hostPort,
+ weakIntern(exec.hostPort),
memoryUsed,
exec.maxMemory - exec.memoryUsed,
diskUsed,
@@ -524,7 +498,9 @@ private class LiveRDDDistribution(exec: LiveExecutor) {
private class LiveRDD(val info: RDDInfo) extends LiveEntity {
- var storageLevel: String = info.storageLevel.description
+ import LiveEntityHelpers._
+
+ var storageLevel: String = weakIntern(info.storageLevel.description)
var memoryUsed = 0L
var diskUsed = 0L
@@ -533,6 +509,10 @@ private class LiveRDD(val info: RDDInfo) extends LiveEntity {
private val distributions = new HashMap[String, LiveRDDDistribution]()
+ def setStorageLevel(level: String): Unit = {
+ this.storageLevel = weakIntern(level)
+ }
+
def partition(blockName: String): LiveRDDPartition = {
partitions.getOrElseUpdate(blockName, {
val part = new LiveRDDPartition(blockName)
@@ -593,6 +573,9 @@ private class SchedulerPool(name: String) extends LiveEntity {
private object LiveEntityHelpers {
+ private val stringInterner = Interners.newWeakInterner[String]()
+
+
def newAccumulatorInfos(accums: Iterable[AccumulableInfo]): Seq[v1.AccumulableInfo] = {
accums
.filter { acc =>
@@ -604,13 +587,119 @@ private object LiveEntityHelpers {
.map { acc =>
new v1.AccumulableInfo(
acc.id,
- acc.name.orNull,
+ acc.name.map(weakIntern).orNull,
acc.update.map(_.toString()),
acc.value.map(_.toString()).orNull)
}
.toSeq
}
+ /** String interning to reduce the memory usage. */
+ def weakIntern(s: String): String = {
+ stringInterner.intern(s)
+ }
+
+ // scalastyle:off argcount
+ def createMetrics(
+ executorDeserializeTime: Long,
+ executorDeserializeCpuTime: Long,
+ executorRunTime: Long,
+ executorCpuTime: Long,
+ resultSize: Long,
+ jvmGcTime: Long,
+ resultSerializationTime: Long,
+ memoryBytesSpilled: Long,
+ diskBytesSpilled: Long,
+ peakExecutionMemory: Long,
+ inputBytesRead: Long,
+ inputRecordsRead: Long,
+ outputBytesWritten: Long,
+ outputRecordsWritten: Long,
+ shuffleRemoteBlocksFetched: Long,
+ shuffleLocalBlocksFetched: Long,
+ shuffleFetchWaitTime: Long,
+ shuffleRemoteBytesRead: Long,
+ shuffleRemoteBytesReadToDisk: Long,
+ shuffleLocalBytesRead: Long,
+ shuffleRecordsRead: Long,
+ shuffleBytesWritten: Long,
+ shuffleWriteTime: Long,
+ shuffleRecordsWritten: Long): v1.TaskMetrics = {
+ new v1.TaskMetrics(
+ executorDeserializeTime,
+ executorDeserializeCpuTime,
+ executorRunTime,
+ executorCpuTime,
+ resultSize,
+ jvmGcTime,
+ resultSerializationTime,
+ memoryBytesSpilled,
+ diskBytesSpilled,
+ peakExecutionMemory,
+ new v1.InputMetrics(
+ inputBytesRead,
+ inputRecordsRead),
+ new v1.OutputMetrics(
+ outputBytesWritten,
+ outputRecordsWritten),
+ new v1.ShuffleReadMetrics(
+ shuffleRemoteBlocksFetched,
+ shuffleLocalBlocksFetched,
+ shuffleFetchWaitTime,
+ shuffleRemoteBytesRead,
+ shuffleRemoteBytesReadToDisk,
+ shuffleLocalBytesRead,
+ shuffleRecordsRead),
+ new v1.ShuffleWriteMetrics(
+ shuffleBytesWritten,
+ shuffleWriteTime,
+ shuffleRecordsWritten))
+ }
+ // scalastyle:on argcount
+
+ def createMetrics(default: Long): v1.TaskMetrics = {
+ createMetrics(default, default, default, default, default, default, default, default,
+ default, default, default, default, default, default, default, default,
+ default, default, default, default, default, default, default, default)
+ }
+
+ /** Add m2 values to m1. */
+ def addMetrics(m1: v1.TaskMetrics, m2: v1.TaskMetrics): v1.TaskMetrics = addMetrics(m1, m2, 1)
+
+ /** Subtract m2 values from m1. */
+ def subtractMetrics(m1: v1.TaskMetrics, m2: v1.TaskMetrics): v1.TaskMetrics = {
+ addMetrics(m1, m2, -1)
+ }
+
+ private def addMetrics(m1: v1.TaskMetrics, m2: v1.TaskMetrics, mult: Int): v1.TaskMetrics = {
+ createMetrics(
+ m1.executorDeserializeTime + m2.executorDeserializeTime * mult,
+ m1.executorDeserializeCpuTime + m2.executorDeserializeCpuTime * mult,
+ m1.executorRunTime + m2.executorRunTime * mult,
+ m1.executorCpuTime + m2.executorCpuTime * mult,
+ m1.resultSize + m2.resultSize * mult,
+ m1.jvmGcTime + m2.jvmGcTime * mult,
+ m1.resultSerializationTime + m2.resultSerializationTime * mult,
+ m1.memoryBytesSpilled + m2.memoryBytesSpilled * mult,
+ m1.diskBytesSpilled + m2.diskBytesSpilled * mult,
+ m1.peakExecutionMemory + m2.peakExecutionMemory * mult,
+ m1.inputMetrics.bytesRead + m2.inputMetrics.bytesRead * mult,
+ m1.inputMetrics.recordsRead + m2.inputMetrics.recordsRead * mult,
+ m1.outputMetrics.bytesWritten + m2.outputMetrics.bytesWritten * mult,
+ m1.outputMetrics.recordsWritten + m2.outputMetrics.recordsWritten * mult,
+ m1.shuffleReadMetrics.remoteBlocksFetched + m2.shuffleReadMetrics.remoteBlocksFetched * mult,
+ m1.shuffleReadMetrics.localBlocksFetched + m2.shuffleReadMetrics.localBlocksFetched * mult,
+ m1.shuffleReadMetrics.fetchWaitTime + m2.shuffleReadMetrics.fetchWaitTime * mult,
+ m1.shuffleReadMetrics.remoteBytesRead + m2.shuffleReadMetrics.remoteBytesRead * mult,
+ m1.shuffleReadMetrics.remoteBytesReadToDisk +
+ m2.shuffleReadMetrics.remoteBytesReadToDisk * mult,
+ m1.shuffleReadMetrics.localBytesRead + m2.shuffleReadMetrics.localBytesRead * mult,
+ m1.shuffleReadMetrics.recordsRead + m2.shuffleReadMetrics.recordsRead * mult,
+ m1.shuffleWriteMetrics.bytesWritten + m2.shuffleWriteMetrics.bytesWritten * mult,
+ m1.shuffleWriteMetrics.writeTime + m2.shuffleWriteMetrics.writeTime * mult,
+ m1.shuffleWriteMetrics.recordsWritten + m2.shuffleWriteMetrics.recordsWritten * mult)
+ }
+
}
/**
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
index ed9bdc6e1e3c..d121068718b8 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
@@ -49,6 +49,7 @@ private[v1] class ApiRootResource extends ApiRequestContext {
@Path("applications/{appId}")
def application(): Class[OneApplicationResource] = classOf[OneApplicationResource]
+ @GET
@Path("version")
def version(): VersionInfo = new VersionInfo(org.apache.spark.SPARK_VERSION)
@@ -157,6 +158,14 @@ private[v1] class NotFoundException(msg: String) extends WebApplicationException
.build()
)
+private[v1] class ServiceUnavailable(msg: String) extends WebApplicationException(
+ new ServiceUnavailableException(msg),
+ Response
+ .status(Response.Status.SERVICE_UNAVAILABLE)
+ .entity(ErrorWrapper(msg))
+ .build()
+)
+
private[v1] class BadParameterException(msg: String) extends WebApplicationException(
new IllegalArgumentException(msg),
Response
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
index bd4df07e7afc..974697890dd0 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
@@ -19,13 +19,13 @@ package org.apache.spark.status.api.v1
import java.io.OutputStream
import java.util.{List => JList}
import java.util.zip.ZipOutputStream
-import javax.ws.rs.{GET, Path, PathParam, Produces, QueryParam}
+import javax.ws.rs._
import javax.ws.rs.core.{MediaType, Response, StreamingOutput}
import scala.util.control.NonFatal
-import org.apache.spark.JobExecutionStatus
-import org.apache.spark.ui.SparkUI
+import org.apache.spark.{JobExecutionStatus, SparkContext}
+import org.apache.spark.ui.UIUtils
@Produces(Array(MediaType.APPLICATION_JSON))
private[v1] class AbstractApplicationResource extends BaseAppResource {
@@ -51,6 +51,29 @@ private[v1] class AbstractApplicationResource extends BaseAppResource {
@Path("executors")
def executorList(): Seq[ExecutorSummary] = withUI(_.store.executorList(true))
+ @GET
+ @Path("executors/{executorId}/threads")
+ def threadDump(@PathParam("executorId") execId: String): Array[ThreadStackTrace] = withUI { ui =>
+ if (execId != SparkContext.DRIVER_IDENTIFIER && !execId.forall(Character.isDigit)) {
+ throw new BadParameterException(
+ s"Invalid executorId: neither '${SparkContext.DRIVER_IDENTIFIER}' nor number.")
+ }
+
+ val safeSparkContext = ui.sc.getOrElse {
+ throw new ServiceUnavailable("Thread dumps not available through the history server.")
+ }
+
+ ui.store.asOption(ui.store.executorSummary(execId)) match {
+ case Some(executorSummary) if executorSummary.isActive =>
+ val safeThreadDump = safeSparkContext.getExecutorThreadDump(execId).getOrElse {
+ throw new NotFoundException("No thread dump is available.")
+ }
+ safeThreadDump
+ case Some(_) => throw new BadParameterException("Executor is not active.")
+ case _ => throw new NotFoundException("Executor does not exist.")
+ }
+ }
+
@GET
@Path("allexecutors")
def allExecutorList(): Seq[ExecutorSummary] = withUI(_.store.executorList(false))
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
index 3b879545b3d2..96249e4bfd5f 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
@@ -87,7 +87,8 @@ private[v1] class StagesResource extends BaseAppResource {
}
}
- ui.store.taskSummary(stageId, stageAttemptId, quantiles)
+ ui.store.taskSummary(stageId, stageAttemptId, quantiles).getOrElse(
+ throw new NotFoundException(s"No tasks reported metrics for $stageId / $stageAttemptId yet."))
}
@GET
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index 45eaf935fb08..971d7e90fa7b 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -19,6 +19,8 @@ package org.apache.spark.status.api.v1
import java.lang.{Long => JLong}
import java.util.Date
+import scala.xml.{NodeSeq, Text}
+
import com.fasterxml.jackson.annotation.JsonIgnoreProperties
import com.fasterxml.jackson.databind.annotation.JsonDeserialize
@@ -68,7 +70,8 @@ class ExecutorStageSummary private[spark](
val shuffleWrite : Long,
val shuffleWriteRecords : Long,
val memoryBytesSpilled : Long,
- val diskBytesSpilled : Long)
+ val diskBytesSpilled : Long,
+ val isBlacklistedForStage: Boolean)
class ExecutorSummary private[spark](
val id: String,
@@ -94,7 +97,8 @@ class ExecutorSummary private[spark](
val removeTime: Option[Date],
val removeReason: Option[String],
val executorLogs: Map[String, String],
- val memoryMetrics: Option[MemoryMetrics])
+ val memoryMetrics: Option[MemoryMetrics],
+ val blacklistedInStages: Set[Int])
class MemoryMetrics private[spark](
val usedOnHeapStorageMemory: Long,
@@ -261,6 +265,9 @@ class TaskMetricDistributions private[spark](
val resultSize: IndexedSeq[Double],
val jvmGcTime: IndexedSeq[Double],
val resultSerializationTime: IndexedSeq[Double],
+ val gettingResultTime: IndexedSeq[Double],
+ val schedulerDelay: IndexedSeq[Double],
+ val peakExecutionMemory: IndexedSeq[Double],
val memoryBytesSpilled: IndexedSeq[Double],
val diskBytesSpilled: IndexedSeq[Double],
@@ -311,3 +318,32 @@ class RuntimeInfo private[spark](
val javaVersion: String,
val javaHome: String,
val scalaVersion: String)
+
+case class StackTrace(elems: Seq[String]) {
+ override def toString: String = elems.mkString
+
+ def html: NodeSeq = {
+ val withNewLine = elems.foldLeft(NodeSeq.Empty) { (acc, elem) =>
+ if (acc.isEmpty) {
+ acc :+ Text(elem)
+ } else {
+ acc :+ :+ Text(elem)
+ }
+ }
+
+ withNewLine
+ }
+
+ def mkString(start: String, sep: String, end: String): String = {
+ elems.mkString(start, sep, end)
+ }
+}
+
+case class ThreadStackTrace(
+ val threadId: Long,
+ val threadName: String,
+ val threadState: Thread.State,
+ val stackTrace: StackTrace,
+ val blockedByThreadId: Option[Long],
+ val blockedByLock: String,
+ val holdingLocks: Seq[String])
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
index 1cfd30df4909..646cf25880e3 100644
--- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -17,9 +17,11 @@
package org.apache.spark.status
-import java.lang.{Integer => JInteger, Long => JLong}
+import java.lang.{Long => JLong}
+import java.util.Date
import com.fasterxml.jackson.annotation.JsonIgnore
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize
import org.apache.spark.status.KVUtils._
import org.apache.spark.status.api.v1._
@@ -49,10 +51,10 @@ private[spark] class ApplicationEnvironmentInfoWrapper(val info: ApplicationEnvi
private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) {
@JsonIgnore @KVIndex
- private[this] val id: String = info.id
+ private def id: String = info.id
@JsonIgnore @KVIndex("active")
- private[this] val active: Boolean = info.isActive
+ private def active: Boolean = info.isActive
@JsonIgnore @KVIndex("host")
val host: String = info.hostPort.split(":")(0)
@@ -69,52 +71,281 @@ private[spark] class JobDataWrapper(
val skippedStages: Set[Int]) {
@JsonIgnore @KVIndex
- private[this] val id: Int = info.jobId
+ private def id: Int = info.jobId
+ @JsonIgnore @KVIndex("completionTime")
+ private def completionTime: Long = info.completionTime.map(_.getTime).getOrElse(-1L)
}
private[spark] class StageDataWrapper(
val info: StageData,
- val jobIds: Set[Int]) {
+ val jobIds: Set[Int],
+ @JsonDeserialize(contentAs = classOf[JLong])
+ val locality: Map[String, Long]) {
@JsonIgnore @KVIndex
- def id: Array[Int] = Array(info.stageId, info.attemptId)
+ private[this] val id: Array[Int] = Array(info.stageId, info.attemptId)
@JsonIgnore @KVIndex("stageId")
- def stageId: Int = info.stageId
+ private def stageId: Int = info.stageId
+ @JsonIgnore @KVIndex("active")
+ private def active: Boolean = info.status == StageStatus.ACTIVE
+
+ @JsonIgnore @KVIndex("completionTime")
+ private def completionTime: Long = info.completionTime.map(_.getTime).getOrElse(-1L)
+}
+
+/**
+ * Tasks have a lot of indices that are used in a few different places. This object keeps logical
+ * names for these indices, mapped to short strings to save space when using a disk store.
+ */
+private[spark] object TaskIndexNames {
+ final val ACCUMULATORS = "acc"
+ final val ATTEMPT = "att"
+ final val DESER_CPU_TIME = "dct"
+ final val DESER_TIME = "des"
+ final val DISK_SPILL = "dbs"
+ final val DURATION = "dur"
+ final val ERROR = "err"
+ final val EXECUTOR = "exe"
+ final val HOST = "hst"
+ final val EXEC_CPU_TIME = "ect"
+ final val EXEC_RUN_TIME = "ert"
+ final val GC_TIME = "gc"
+ final val GETTING_RESULT_TIME = "grt"
+ final val INPUT_RECORDS = "ir"
+ final val INPUT_SIZE = "is"
+ final val LAUNCH_TIME = "lt"
+ final val LOCALITY = "loc"
+ final val MEM_SPILL = "mbs"
+ final val OUTPUT_RECORDS = "or"
+ final val OUTPUT_SIZE = "os"
+ final val PEAK_MEM = "pem"
+ final val RESULT_SIZE = "rs"
+ final val SCHEDULER_DELAY = "dly"
+ final val SER_TIME = "rst"
+ final val SHUFFLE_LOCAL_BLOCKS = "slbl"
+ final val SHUFFLE_READ_RECORDS = "srr"
+ final val SHUFFLE_READ_TIME = "srt"
+ final val SHUFFLE_REMOTE_BLOCKS = "srbl"
+ final val SHUFFLE_REMOTE_READS = "srby"
+ final val SHUFFLE_REMOTE_READS_TO_DISK = "srbd"
+ final val SHUFFLE_TOTAL_READS = "stby"
+ final val SHUFFLE_TOTAL_BLOCKS = "stbl"
+ final val SHUFFLE_WRITE_RECORDS = "swr"
+ final val SHUFFLE_WRITE_SIZE = "sws"
+ final val SHUFFLE_WRITE_TIME = "swt"
+ final val STAGE = "stage"
+ final val STATUS = "sta"
+ final val TASK_INDEX = "idx"
+ final val COMPLETION_TIME = "ct"
}
/**
- * The task information is always indexed with the stage ID, since that is how the UI and API
- * consume it. That means every indexed value has the stage ID and attempt ID included, aside
- * from the actual data being indexed.
+ * Unlike other data types, the task data wrapper does not keep a reference to the API's TaskData.
+ * That is to save memory, since for large applications there can be a large number of these
+ * elements (by default up to 100,000 per stage), and every bit of wasted memory adds up.
+ *
+ * It also contains many secondary indices, which are used to sort data efficiently in the UI at the
+ * expense of storage space (and slower write times).
*/
private[spark] class TaskDataWrapper(
- val info: TaskData,
+ // Storing this as an object actually saves memory; it's also used as the key in the in-memory
+ // store, so in that case you'd save the extra copy of the value here.
+ @KVIndexParam
+ val taskId: JLong,
+ @KVIndexParam(value = TaskIndexNames.TASK_INDEX, parent = TaskIndexNames.STAGE)
+ val index: Int,
+ @KVIndexParam(value = TaskIndexNames.ATTEMPT, parent = TaskIndexNames.STAGE)
+ val attempt: Int,
+ @KVIndexParam(value = TaskIndexNames.LAUNCH_TIME, parent = TaskIndexNames.STAGE)
+ val launchTime: Long,
+ val resultFetchStart: Long,
+ @KVIndexParam(value = TaskIndexNames.DURATION, parent = TaskIndexNames.STAGE)
+ val duration: Long,
+ @KVIndexParam(value = TaskIndexNames.EXECUTOR, parent = TaskIndexNames.STAGE)
+ val executorId: String,
+ @KVIndexParam(value = TaskIndexNames.HOST, parent = TaskIndexNames.STAGE)
+ val host: String,
+ @KVIndexParam(value = TaskIndexNames.STATUS, parent = TaskIndexNames.STAGE)
+ val status: String,
+ @KVIndexParam(value = TaskIndexNames.LOCALITY, parent = TaskIndexNames.STAGE)
+ val taskLocality: String,
+ val speculative: Boolean,
+ val accumulatorUpdates: Seq[AccumulableInfo],
+ val errorMessage: Option[String],
+
+ // The following is an exploded view of a TaskMetrics API object. This saves 5 objects
+ // (= 80 bytes of Java object overhead) per instance of this wrapper. If the first value
+ // (executorDeserializeTime) is -1L, it means the metrics for this task have not been
+ // recorded.
+ @KVIndexParam(value = TaskIndexNames.DESER_TIME, parent = TaskIndexNames.STAGE)
+ val executorDeserializeTime: Long,
+ @KVIndexParam(value = TaskIndexNames.DESER_CPU_TIME, parent = TaskIndexNames.STAGE)
+ val executorDeserializeCpuTime: Long,
+ @KVIndexParam(value = TaskIndexNames.EXEC_RUN_TIME, parent = TaskIndexNames.STAGE)
+ val executorRunTime: Long,
+ @KVIndexParam(value = TaskIndexNames.EXEC_CPU_TIME, parent = TaskIndexNames.STAGE)
+ val executorCpuTime: Long,
+ @KVIndexParam(value = TaskIndexNames.RESULT_SIZE, parent = TaskIndexNames.STAGE)
+ val resultSize: Long,
+ @KVIndexParam(value = TaskIndexNames.GC_TIME, parent = TaskIndexNames.STAGE)
+ val jvmGcTime: Long,
+ @KVIndexParam(value = TaskIndexNames.SER_TIME, parent = TaskIndexNames.STAGE)
+ val resultSerializationTime: Long,
+ @KVIndexParam(value = TaskIndexNames.MEM_SPILL, parent = TaskIndexNames.STAGE)
+ val memoryBytesSpilled: Long,
+ @KVIndexParam(value = TaskIndexNames.DISK_SPILL, parent = TaskIndexNames.STAGE)
+ val diskBytesSpilled: Long,
+ @KVIndexParam(value = TaskIndexNames.PEAK_MEM, parent = TaskIndexNames.STAGE)
+ val peakExecutionMemory: Long,
+ @KVIndexParam(value = TaskIndexNames.INPUT_SIZE, parent = TaskIndexNames.STAGE)
+ val inputBytesRead: Long,
+ @KVIndexParam(value = TaskIndexNames.INPUT_RECORDS, parent = TaskIndexNames.STAGE)
+ val inputRecordsRead: Long,
+ @KVIndexParam(value = TaskIndexNames.OUTPUT_SIZE, parent = TaskIndexNames.STAGE)
+ val outputBytesWritten: Long,
+ @KVIndexParam(value = TaskIndexNames.OUTPUT_RECORDS, parent = TaskIndexNames.STAGE)
+ val outputRecordsWritten: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_REMOTE_BLOCKS, parent = TaskIndexNames.STAGE)
+ val shuffleRemoteBlocksFetched: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_LOCAL_BLOCKS, parent = TaskIndexNames.STAGE)
+ val shuffleLocalBlocksFetched: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_READ_TIME, parent = TaskIndexNames.STAGE)
+ val shuffleFetchWaitTime: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_REMOTE_READS, parent = TaskIndexNames.STAGE)
+ val shuffleRemoteBytesRead: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_REMOTE_READS_TO_DISK,
+ parent = TaskIndexNames.STAGE)
+ val shuffleRemoteBytesReadToDisk: Long,
+ val shuffleLocalBytesRead: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_READ_RECORDS, parent = TaskIndexNames.STAGE)
+ val shuffleRecordsRead: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_WRITE_SIZE, parent = TaskIndexNames.STAGE)
+ val shuffleBytesWritten: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_WRITE_TIME, parent = TaskIndexNames.STAGE)
+ val shuffleWriteTime: Long,
+ @KVIndexParam(value = TaskIndexNames.SHUFFLE_WRITE_RECORDS, parent = TaskIndexNames.STAGE)
+ val shuffleRecordsWritten: Long,
+
val stageId: Int,
val stageAttemptId: Int) {
- @JsonIgnore @KVIndex
- def id: Long = info.taskId
+ def hasMetrics: Boolean = executorDeserializeTime >= 0
+
+ def toApi: TaskData = {
+ val metrics = if (hasMetrics) {
+ Some(new TaskMetrics(
+ executorDeserializeTime,
+ executorDeserializeCpuTime,
+ executorRunTime,
+ executorCpuTime,
+ resultSize,
+ jvmGcTime,
+ resultSerializationTime,
+ memoryBytesSpilled,
+ diskBytesSpilled,
+ peakExecutionMemory,
+ new InputMetrics(
+ inputBytesRead,
+ inputRecordsRead),
+ new OutputMetrics(
+ outputBytesWritten,
+ outputRecordsWritten),
+ new ShuffleReadMetrics(
+ shuffleRemoteBlocksFetched,
+ shuffleLocalBlocksFetched,
+ shuffleFetchWaitTime,
+ shuffleRemoteBytesRead,
+ shuffleRemoteBytesReadToDisk,
+ shuffleLocalBytesRead,
+ shuffleRecordsRead),
+ new ShuffleWriteMetrics(
+ shuffleBytesWritten,
+ shuffleWriteTime,
+ shuffleRecordsWritten)))
+ } else {
+ None
+ }
- @JsonIgnore @KVIndex("stage")
- def stage: Array[Int] = Array(stageId, stageAttemptId)
+ new TaskData(
+ taskId,
+ index,
+ attempt,
+ new Date(launchTime),
+ if (resultFetchStart > 0L) Some(new Date(resultFetchStart)) else None,
+ if (duration > 0L) Some(duration) else None,
+ executorId,
+ host,
+ status,
+ taskLocality,
+ speculative,
+ accumulatorUpdates,
+ errorMessage,
+ metrics)
+ }
+
+ @JsonIgnore @KVIndex(TaskIndexNames.STAGE)
+ private def stage: Array[Int] = Array(stageId, stageAttemptId)
- @JsonIgnore @KVIndex("runtime")
- def runtime: Array[AnyRef] = {
- val _runtime = info.taskMetrics.map(_.executorRunTime).getOrElse(-1L)
- Array(stageId: JInteger, stageAttemptId: JInteger, _runtime: JLong)
+ @JsonIgnore @KVIndex(value = TaskIndexNames.SCHEDULER_DELAY, parent = TaskIndexNames.STAGE)
+ def schedulerDelay: Long = {
+ if (hasMetrics) {
+ AppStatusUtils.schedulerDelay(launchTime, resultFetchStart, duration, executorDeserializeTime,
+ resultSerializationTime, executorRunTime)
+ } else {
+ -1L
+ }
}
- @JsonIgnore @KVIndex("startTime")
- def startTime: Array[AnyRef] = {
- Array(stageId: JInteger, stageAttemptId: JInteger, info.launchTime.getTime(): JLong)
+ @JsonIgnore @KVIndex(value = TaskIndexNames.GETTING_RESULT_TIME, parent = TaskIndexNames.STAGE)
+ def gettingResultTime: Long = {
+ if (hasMetrics) {
+ AppStatusUtils.gettingResultTime(launchTime, resultFetchStart, duration)
+ } else {
+ -1L
+ }
}
- @JsonIgnore @KVIndex("active")
- def active: Boolean = info.duration.isEmpty
+ /**
+ * Sorting by accumulators is a little weird, and the previous behavior would generate
+ * insanely long keys in the index. So this implementation just considers the first
+ * accumulator and its String representation.
+ */
+ @JsonIgnore @KVIndex(value = TaskIndexNames.ACCUMULATORS, parent = TaskIndexNames.STAGE)
+ private def accumulators: String = {
+ if (accumulatorUpdates.nonEmpty) {
+ val acc = accumulatorUpdates.head
+ s"${acc.name}:${acc.value}"
+ } else {
+ ""
+ }
+ }
+
+ @JsonIgnore @KVIndex(value = TaskIndexNames.SHUFFLE_TOTAL_READS, parent = TaskIndexNames.STAGE)
+ private def shuffleTotalReads: Long = {
+ if (hasMetrics) {
+ shuffleLocalBytesRead + shuffleRemoteBytesRead
+ } else {
+ -1L
+ }
+ }
+
+ @JsonIgnore @KVIndex(value = TaskIndexNames.SHUFFLE_TOTAL_BLOCKS, parent = TaskIndexNames.STAGE)
+ private def shuffleTotalBlocks: Long = {
+ if (hasMetrics) {
+ shuffleLocalBlocksFetched + shuffleRemoteBlocksFetched
+ } else {
+ -1L
+ }
+ }
+
+ @JsonIgnore @KVIndex(value = TaskIndexNames.ERROR, parent = TaskIndexNames.STAGE)
+ private def error: String = if (errorMessage.isDefined) errorMessage.get else ""
+ @JsonIgnore @KVIndex(value = TaskIndexNames.COMPLETION_TIME, parent = TaskIndexNames.STAGE)
+ private def completionTime: Long = launchTime + duration
}
private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) {
@@ -134,10 +365,13 @@ private[spark] class ExecutorStageSummaryWrapper(
val info: ExecutorStageSummary) {
@JsonIgnore @KVIndex
- val id: Array[Any] = Array(stageId, stageAttemptId, executorId)
+ private val _id: Array[Any] = Array(stageId, stageAttemptId, executorId)
@JsonIgnore @KVIndex("stage")
- private[this] val stage: Array[Int] = Array(stageId, stageAttemptId)
+ private def stage: Array[Int] = Array(stageId, stageAttemptId)
+
+ @JsonIgnore
+ def id: Array[Any] = _id
}
@@ -203,3 +437,53 @@ private[spark] class AppSummary(
def id: String = classOf[AppSummary].getName()
}
+
+/**
+ * A cached view of a specific quantile for one stage attempt's metrics.
+ */
+private[spark] class CachedQuantile(
+ val stageId: Int,
+ val stageAttemptId: Int,
+ val quantile: String,
+ val taskCount: Long,
+
+ // The following fields are an exploded view of a single entry for TaskMetricDistributions.
+ val executorDeserializeTime: Double,
+ val executorDeserializeCpuTime: Double,
+ val executorRunTime: Double,
+ val executorCpuTime: Double,
+ val resultSize: Double,
+ val jvmGcTime: Double,
+ val resultSerializationTime: Double,
+ val gettingResultTime: Double,
+ val schedulerDelay: Double,
+ val peakExecutionMemory: Double,
+ val memoryBytesSpilled: Double,
+ val diskBytesSpilled: Double,
+
+ val bytesRead: Double,
+ val recordsRead: Double,
+
+ val bytesWritten: Double,
+ val recordsWritten: Double,
+
+ val shuffleReadBytes: Double,
+ val shuffleRecordsRead: Double,
+ val shuffleRemoteBlocksFetched: Double,
+ val shuffleLocalBlocksFetched: Double,
+ val shuffleFetchWaitTime: Double,
+ val shuffleRemoteBytesRead: Double,
+ val shuffleRemoteBytesReadToDisk: Double,
+ val shuffleTotalBlocksFetched: Double,
+
+ val shuffleWriteBytes: Double,
+ val shuffleWriteRecords: Double,
+ val shuffleWriteTime: Double) {
+
+ @KVIndex @JsonIgnore
+ def id: Array[Any] = Array(stageId, stageAttemptId, quantile)
+
+ @KVIndex("stage") @JsonIgnore
+ def stage: Array[Int] = Array(stageId, stageAttemptId)
+
+}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
index 2c3da0ee85e0..d4a59c33b974 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
@@ -18,7 +18,8 @@
package org.apache.spark.storage
import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
-import java.util.concurrent.ConcurrentHashMap
+
+import com.google.common.cache.{CacheBuilder, CacheLoader}
import org.apache.spark.SparkContext
import org.apache.spark.annotation.DeveloperApi
@@ -132,10 +133,17 @@ private[spark] object BlockManagerId {
getCachedBlockManagerId(obj)
}
- val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]()
+ /**
+ * The max cache size is hardcoded to 10000, since the size of a BlockManagerId
+ * object is about 48B, the total memory cost should be below 1MB which is feasible.
+ */
+ val blockManagerIdCache = CacheBuilder.newBuilder()
+ .maximumSize(10000)
+ .build(new CacheLoader[BlockManagerId, BlockManagerId]() {
+ override def load(id: BlockManagerId) = id
+ })
def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = {
- blockManagerIdCache.putIfAbsent(id, id)
blockManagerIdCache.get(id)
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
index 89a6a71a589a..8e8f7d197c9e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
@@ -164,7 +164,8 @@ class BlockManagerMasterEndpoint(
val futures = blockManagerInfo.values.map { bm =>
bm.slaveEndpoint.ask[Int](removeMsg).recover {
case e: IOException =>
- logWarning(s"Error trying to remove RDD $rddId", e)
+ logWarning(s"Error trying to remove RDD $rddId from block manager ${bm.blockManagerId}",
+ e)
0 // zero blocks were removed
}
}.toSeq
@@ -192,11 +193,16 @@ class BlockManagerMasterEndpoint(
val requiredBlockManagers = blockManagerInfo.values.filter { info =>
removeFromDriver || !info.blockManagerId.isDriver
}
- Future.sequence(
- requiredBlockManagers.map { bm =>
- bm.slaveEndpoint.ask[Int](removeMsg)
- }.toSeq
- )
+ val futures = requiredBlockManagers.map { bm =>
+ bm.slaveEndpoint.ask[Int](removeMsg).recover {
+ case e: IOException =>
+ logWarning(s"Error trying to remove broadcast $broadcastId from block manager " +
+ s"${bm.blockManagerId}", e)
+ 0 // zero blocks were removed
+ }
+ }.toSeq
+
+ Future.sequence(futures)
}
private def removeBlockManager(blockManagerId: BlockManagerId) {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala
index 353eac60df17..0bacc34cdfd9 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala
@@ -54,10 +54,9 @@ trait BlockReplicationPolicy {
}
object BlockReplicationUtils {
- // scalastyle:off line.size.limit
/**
* Uses sampling algorithm by Robert Floyd. Finds a random sample in O(n) while
- * minimizing space usage. Please see
+ * minimizing space usage. Please see
* here.
*
* @param n total number of indices
@@ -65,7 +64,6 @@ object BlockReplicationUtils {
* @param r random number generator
* @return list of m random unique indices
*/
- // scalastyle:on line.size.limit
private def getSampleIds(n: Int, m: Int, r: Random): List[Int] = {
val indices = (n - m + 1 to n).foldLeft(mutable.LinkedHashSet.empty[Int]) {case (set, i) =>
val t = r.nextInt(i) + 1
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index 98b5a735a452..b31862323a89 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -48,7 +48,9 @@ import org.apache.spark.util.io.ChunkedByteBufferOutputStream
* @param blockManager [[BlockManager]] for reading local blocks
* @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]].
* For each block we also require the size (in bytes as a long field) in
- * order to throttle the memory usage.
+ * order to throttle the memory usage. Note that zero-sized blocks are
+ * already excluded, which happened in
+ * [[MapOutputTracker.convertMapStatuses]].
* @param streamWrapper A function to wrap the returned input stream.
* @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point.
* @param maxReqsInFlight max number of remote requests to fetch blocks at any given point.
@@ -62,7 +64,7 @@ final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: ShuffleClient,
blockManager: BlockManager,
- blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
+ blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
@@ -74,8 +76,8 @@ final class ShuffleBlockFetcherIterator(
import ShuffleBlockFetcherIterator._
/**
- * Total number of blocks to fetch. This can be smaller than the total number of blocks
- * in [[blocksByAddress]] because we filter out zero-sized blocks in [[initialize]].
+ * Total number of blocks to fetch. This should be equal to the total number of blocks
+ * in [[blocksByAddress]] because we already filter out zero-sized blocks in [[blocksByAddress]].
*
* This should equal localBlocks.size + remoteBlocks.size.
*/
@@ -90,7 +92,7 @@ final class ShuffleBlockFetcherIterator(
private[this] val startTime = System.currentTimeMillis
/** Local blocks to fetch, excluding zero-sized blocks. */
- private[this] val localBlocks = new ArrayBuffer[BlockId]()
+ private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]()
/** Remote blocks to fetch, excluding zero-sized blocks. */
private[this] val remoteBlocks = new HashSet[BlockId]()
@@ -267,13 +269,16 @@ final class ShuffleBlockFetcherIterator(
// at most maxBytesInFlight in order to limit the amount of data in flight.
val remoteRequests = new ArrayBuffer[FetchRequest]
- // Tracks total number of blocks (including zero sized blocks)
- var totalBlocks = 0
for ((address, blockInfos) <- blocksByAddress) {
- totalBlocks += blockInfos.size
if (address.executorId == blockManager.blockManagerId.executorId) {
- // Filter out zero-sized blocks
- localBlocks ++= blockInfos.filter(_._2 != 0).map(_._1)
+ blockInfos.find(_._2 <= 0) match {
+ case Some((blockId, size)) if size < 0 =>
+ throw new BlockException(blockId, "Negative block size " + size)
+ case Some((blockId, size)) if size == 0 =>
+ throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
+ case None => // do nothing.
+ }
+ localBlocks ++= blockInfos.map(_._1)
numBlocksToFetch += localBlocks.size
} else {
val iterator = blockInfos.iterator
@@ -281,14 +286,15 @@ final class ShuffleBlockFetcherIterator(
var curBlocks = new ArrayBuffer[(BlockId, Long)]
while (iterator.hasNext) {
val (blockId, size) = iterator.next()
- // Skip empty blocks
- if (size > 0) {
+ if (size < 0) {
+ throw new BlockException(blockId, "Negative block size " + size)
+ } else if (size == 0) {
+ throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
+ } else {
curBlocks += ((blockId, size))
remoteBlocks += blockId
numBlocksToFetch += 1
curRequestSize += size
- } else if (size < 0) {
- throw new BlockException(blockId, "Negative block size " + size)
}
if (curRequestSize >= targetRequestSize ||
curBlocks.size >= maxBlocksInFlightPerAddress) {
@@ -306,7 +312,8 @@ final class ShuffleBlockFetcherIterator(
}
}
}
- logInfo(s"Getting $numBlocksToFetch non-empty blocks out of $totalBlocks blocks")
+ logInfo(s"Getting $numBlocksToFetch non-empty blocks including ${localBlocks.size}" +
+ s" local blocks and ${remoteBlocks.size} remote blocks")
remoteRequests
}
@@ -316,6 +323,7 @@ final class ShuffleBlockFetcherIterator(
* track in-memory are the ManagedBuffer references themselves.
*/
private[this] def fetchLocalBlocks() {
+ logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}")
val iter = localBlocks.iterator
while (iter.hasNext) {
val blockId = iter.next()
@@ -324,7 +332,8 @@ final class ShuffleBlockFetcherIterator(
shuffleMetrics.incLocalBlocksFetched(1)
shuffleMetrics.incLocalBytesRead(buf.size)
buf.retain()
- results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, 0, buf, false))
+ results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId,
+ buf.size(), buf, false))
} catch {
case e: Exception =>
// If we see an exception, stop immediately.
@@ -397,12 +406,33 @@ final class ShuffleBlockFetcherIterator(
}
shuffleMetrics.incRemoteBlocksFetched(1)
}
- bytesInFlight -= size
+ if (!localBlocks.contains(blockId)) {
+ bytesInFlight -= size
+ }
if (isNetworkReqDone) {
reqsInFlight -= 1
logDebug("Number of requests in flight " + reqsInFlight)
}
+ if (buf.size == 0) {
+ // We will never legitimately receive a zero-size block. All blocks with zero records
+ // have zero size and all zero-size blocks have no records (and hence should never
+ // have been requested in the first place). This statement relies on behaviors of the
+ // shuffle writers, which are guaranteed by the following test cases:
+ //
+ // - BypassMergeSortShuffleWriterSuite: "write with some empty partitions"
+ // - UnsafeShuffleWriterSuite: "writeEmptyIterator"
+ // - DiskBlockObjectWriterSuite: "commit() and close() without ever opening or writing"
+ //
+ // There is not an explicit test for SortShuffleWriter but the underlying APIs that
+ // uses are shared by the UnsafeShuffleWriter (both writers use DiskBlockObjectWriter
+ // which returns a zero-size from commitAndGet() in case no records were written
+ // since the last call.
+ val msg = s"Received a zero-size buffer for block $blockId from $address " +
+ s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)"
+ throwFetchFailedException(blockId, address, new IOException(msg))
+ }
+
val in = try {
buf.createInputStream()
} catch {
@@ -583,8 +613,8 @@ object ShuffleBlockFetcherIterator {
* Result of a fetch from a remote block successfully.
* @param blockId block id
* @param address BlockManager that the block was fetched from.
- * @param size estimated size of the block, used to calculate bytesInFlight.
- * Note that this is NOT the exact bytes.
+ * @param size estimated size of the block. Note that this is NOT the exact bytes.
+ * Size of remote block is used to calculate bytesInFlight.
* @param buf `ManagedBuffer` for the content.
* @param isNetworkReqDone Is this the last network request for this host in this fetch request.
*/
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index e9694fdbca2d..adc406bb1c44 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -24,19 +24,15 @@ import scala.collection.mutable
import sun.nio.ch.DirectBuffer
-import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.internal.Logging
/**
- * :: DeveloperApi ::
* Storage information for each BlockManager.
*
* This class assumes BlockId and BlockStatus are immutable, such that the consumers of this
* class cannot mutate the source of the information. Accesses are not thread-safe.
*/
-@DeveloperApi
-@deprecated("This class may be removed or made private in a future release.", "2.2.0")
-class StorageStatus(
+private[spark] class StorageStatus(
val blockManagerId: BlockManagerId,
val maxMemory: Long,
val maxOnHeapMem: Option[Long],
@@ -44,9 +40,6 @@ class StorageStatus(
/**
* Internal representation of the blocks stored in this block manager.
- *
- * We store RDD blocks and non-RDD blocks separately to allow quick retrievals of RDD blocks.
- * These collections should only be mutated through the add/update/removeBlock methods.
*/
private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]]
private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus]
@@ -87,9 +80,6 @@ class StorageStatus(
*/
def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks }
- /** Return the blocks that belong to the given RDD stored in this block manager. */
- def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = _rddBlocks.getOrElse(rddId, Map.empty)
-
/** Add the given block to this storage status. If it already exists, overwrite it. */
private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = {
updateStorageInfo(blockId, blockStatus)
@@ -101,46 +91,6 @@ class StorageStatus(
}
}
- /** Update the given block in this storage status. If it doesn't already exist, add it. */
- private[spark] def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = {
- addBlock(blockId, blockStatus)
- }
-
- /** Remove the given block from this storage status. */
- private[spark] def removeBlock(blockId: BlockId): Option[BlockStatus] = {
- updateStorageInfo(blockId, BlockStatus.empty)
- blockId match {
- case RDDBlockId(rddId, _) =>
- // Actually remove the block, if it exists
- if (_rddBlocks.contains(rddId)) {
- val removed = _rddBlocks(rddId).remove(blockId)
- // If the given RDD has no more blocks left, remove the RDD
- if (_rddBlocks(rddId).isEmpty) {
- _rddBlocks.remove(rddId)
- }
- removed
- } else {
- None
- }
- case _ =>
- _nonRddBlocks.remove(blockId)
- }
- }
-
- /**
- * Return whether the given block is stored in this block manager in O(1) time.
- *
- * @note This is much faster than `this.blocks.contains`, which is O(blocks) time.
- */
- def containsBlock(blockId: BlockId): Boolean = {
- blockId match {
- case RDDBlockId(rddId, _) =>
- _rddBlocks.get(rddId).exists(_.contains(blockId))
- case _ =>
- _nonRddBlocks.contains(blockId)
- }
- }
-
/**
* Return the given block stored in this block manager in O(1) time.
*
@@ -155,37 +105,12 @@ class StorageStatus(
}
}
- /**
- * Return the number of blocks stored in this block manager in O(RDDs) time.
- *
- * @note This is much faster than `this.blocks.size`, which is O(blocks) time.
- */
- def numBlocks: Int = _nonRddBlocks.size + numRddBlocks
-
- /**
- * Return the number of RDD blocks stored in this block manager in O(RDDs) time.
- *
- * @note This is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time.
- */
- def numRddBlocks: Int = _rddBlocks.values.map(_.size).sum
-
- /**
- * Return the number of blocks that belong to the given RDD in O(1) time.
- *
- * @note This is much faster than `this.rddBlocksById(rddId).size`, which is
- * O(blocks in this RDD) time.
- */
- def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0)
-
/** Return the max memory can be used by this block manager. */
def maxMem: Long = maxMemory
/** Return the memory remaining in this block manager. */
def memRemaining: Long = maxMem - memUsed
- /** Return the memory used by caching RDDs */
- def cacheSize: Long = onHeapCacheSize.getOrElse(0L) + offHeapCacheSize.getOrElse(0L)
-
/** Return the memory used by this block manager. */
def memUsed: Long = onHeapMemUsed.getOrElse(0L) + offHeapMemUsed.getOrElse(0L)
@@ -220,15 +145,9 @@ class StorageStatus(
/** Return the disk space used by this block manager. */
def diskUsed: Long = _nonRddStorageInfo.diskUsage + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum
- /** Return the memory used by the given RDD in this block manager in O(1) time. */
- def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.memoryUsage).getOrElse(0L)
-
/** Return the disk space used by the given RDD in this block manager in O(1) time. */
def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.diskUsage).getOrElse(0L)
- /** Return the storage level, if any, used by the given RDD in this block manager. */
- def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_.level)
-
/**
* Update the relevant storage info, taking into account any existing status for this block.
*/
@@ -295,40 +214,4 @@ private[spark] object StorageUtils extends Logging {
cleaner.clean()
}
}
-
- /**
- * Update the given list of RDDInfo with the given list of storage statuses.
- * This method overwrites the old values stored in the RDDInfo's.
- */
- def updateRddInfo(rddInfos: Seq[RDDInfo], statuses: Seq[StorageStatus]): Unit = {
- rddInfos.foreach { rddInfo =>
- val rddId = rddInfo.id
- // Assume all blocks belonging to the same RDD have the same storage level
- val storageLevel = statuses
- .flatMap(_.rddStorageLevel(rddId)).headOption.getOrElse(StorageLevel.NONE)
- val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum
- val memSize = statuses.map(_.memUsedByRdd(rddId)).sum
- val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum
-
- rddInfo.storageLevel = storageLevel
- rddInfo.numCachedPartitions = numCachedPartitions
- rddInfo.memSize = memSize
- rddInfo.diskSize = diskSize
- }
- }
-
- /**
- * Return a mapping from block ID to its locations for each block that belongs to the given RDD.
- */
- def getRddBlockLocations(rddId: Int, statuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = {
- val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]]
- statuses.foreach { status =>
- status.rddBlocksById(rddId).foreach { case (bid, _) =>
- val location = status.blockManagerId.hostPort
- blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location
- }
- }
- blockLocations
- }
-
}
diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
index 17f7a69ad6ba..4cc5bcb7f9ba 100644
--- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
@@ -162,7 +162,7 @@ private[spark] class MemoryStore(
}
/**
- * Attempt to put the given block in memory store as values.
+ * Attempt to put the given block in memory store as values or bytes.
*
* It's possible that the iterator is too large to materialize and store in memory. To avoid
* OOM exceptions, this method will gradually unroll the iterator while periodically checking
@@ -170,18 +170,24 @@ private[spark] class MemoryStore(
* temporary unroll memory used during the materialization is "transferred" to storage memory,
* so we won't acquire more memory than is actually needed to store the block.
*
- * @return in case of success, the estimated size of the stored data. In case of failure, return
- * an iterator containing the values of the block. The returned iterator will be backed
- * by the combination of the partially-unrolled block and the remaining elements of the
- * original input iterator. The caller must either fully consume this iterator or call
- * `close()` on it in order to free the storage memory consumed by the partially-unrolled
- * block.
+ * @param blockId The block id.
+ * @param values The values which need be stored.
+ * @param classTag the [[ClassTag]] for the block.
+ * @param memoryMode The values saved memory mode(ON_HEAP or OFF_HEAP).
+ * @param valuesHolder A holder that supports storing record of values into memory store as
+ * values or bytes.
+ * @return if the block is stored successfully, return the stored data size. Else return the
+ * memory has reserved for unrolling the block (There are two reasons for store failed:
+ * First, the block is partially-unrolled; second, the block is entirely unrolled and
+ * the actual stored data size is larger than reserved, but we can't request extra
+ * memory).
*/
- private[storage] def putIteratorAsValues[T](
+ private def putIterator[T](
blockId: BlockId,
values: Iterator[T],
- classTag: ClassTag[T]): Either[PartiallyUnrolledIterator[T], Long] = {
-
+ classTag: ClassTag[T],
+ memoryMode: MemoryMode,
+ valuesHolder: ValuesHolder[T]): Either[Long, Long] = {
require(!contains(blockId), s"Block $blockId is already present in the MemoryStore")
// Number of elements unrolled so far
@@ -198,12 +204,10 @@ private[spark] class MemoryStore(
val memoryGrowthFactor = conf.get(UNROLL_MEMORY_GROWTH_FACTOR)
// Keep track of unroll memory used by this particular block / putIterator() operation
var unrollMemoryUsedByThisBlock = 0L
- // Underlying vector for unrolling the block
- var vector = new SizeTrackingVector[T]()(classTag)
// Request enough memory to begin unrolling
keepUnrolling =
- reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, MemoryMode.ON_HEAP)
+ reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, memoryMode)
if (!keepUnrolling) {
logWarning(s"Failed to reserve initial memory threshold of " +
@@ -214,14 +218,14 @@ private[spark] class MemoryStore(
// Unroll this block safely, checking whether we have exceeded our threshold periodically
while (values.hasNext && keepUnrolling) {
- vector += values.next()
+ valuesHolder.storeValue(values.next())
if (elementsUnrolled % memoryCheckPeriod == 0) {
+ val currentSize = valuesHolder.estimatedSize()
// If our vector's size has exceeded the threshold, request more memory
- val currentSize = vector.estimateSize()
if (currentSize >= memoryThreshold) {
val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong
keepUnrolling =
- reserveUnrollMemoryForThisTask(blockId, amountToRequest, MemoryMode.ON_HEAP)
+ reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
if (keepUnrolling) {
unrollMemoryUsedByThisBlock += amountToRequest
}
@@ -232,78 +236,86 @@ private[spark] class MemoryStore(
elementsUnrolled += 1
}
+ // Make sure that we have enough memory to store the block. By this point, it is possible that
+ // the block's actual memory usage has exceeded the unroll memory by a small amount, so we
+ // perform one final call to attempt to allocate additional memory if necessary.
if (keepUnrolling) {
- // We successfully unrolled the entirety of this block
- val arrayValues = vector.toArray
- vector = null
- val entry =
- new DeserializedMemoryEntry[T](arrayValues, SizeEstimator.estimate(arrayValues), classTag)
- val size = entry.size
- def transferUnrollToStorage(amount: Long): Unit = {
+ val entryBuilder = valuesHolder.getBuilder()
+ val size = entryBuilder.preciseSize
+ if (size > unrollMemoryUsedByThisBlock) {
+ val amountToRequest = size - unrollMemoryUsedByThisBlock
+ keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
+ if (keepUnrolling) {
+ unrollMemoryUsedByThisBlock += amountToRequest
+ }
+ }
+
+ if (keepUnrolling) {
+ val entry = entryBuilder.build()
// Synchronize so that transfer is atomic
memoryManager.synchronized {
- releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, amount)
- val success = memoryManager.acquireStorageMemory(blockId, amount, MemoryMode.ON_HEAP)
+ releaseUnrollMemoryForThisTask(memoryMode, unrollMemoryUsedByThisBlock)
+ val success = memoryManager.acquireStorageMemory(blockId, entry.size, memoryMode)
assert(success, "transferring unroll memory to storage memory failed")
}
- }
- // Acquire storage memory if necessary to store this block in memory.
- val enoughStorageMemory = {
- if (unrollMemoryUsedByThisBlock <= size) {
- val acquiredExtra =
- memoryManager.acquireStorageMemory(
- blockId, size - unrollMemoryUsedByThisBlock, MemoryMode.ON_HEAP)
- if (acquiredExtra) {
- transferUnrollToStorage(unrollMemoryUsedByThisBlock)
- }
- acquiredExtra
- } else { // unrollMemoryUsedByThisBlock > size
- // If this task attempt already owns more unroll memory than is necessary to store the
- // block, then release the extra memory that will not be used.
- val excessUnrollMemory = unrollMemoryUsedByThisBlock - size
- releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, excessUnrollMemory)
- transferUnrollToStorage(size)
- true
- }
- }
- if (enoughStorageMemory) {
+
entries.synchronized {
entries.put(blockId, entry)
}
- logInfo("Block %s stored as values in memory (estimated size %s, free %s)".format(
- blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed)))
- Right(size)
+
+ logInfo("Block %s stored as values in memory (estimated size %s, free %s)".format(blockId,
+ Utils.bytesToString(entry.size), Utils.bytesToString(maxMemory - blocksMemoryUsed)))
+ Right(entry.size)
} else {
- assert(currentUnrollMemoryForThisTask >= unrollMemoryUsedByThisBlock,
- "released too much unroll memory")
+ // We ran out of space while unrolling the values for this block
+ logUnrollFailureMessage(blockId, entryBuilder.preciseSize)
+ Left(unrollMemoryUsedByThisBlock)
+ }
+ } else {
+ // We ran out of space while unrolling the values for this block
+ logUnrollFailureMessage(blockId, valuesHolder.estimatedSize())
+ Left(unrollMemoryUsedByThisBlock)
+ }
+ }
+
+ /**
+ * Attempt to put the given block in memory store as values.
+ *
+ * @return in case of success, the estimated size of the stored data. In case of failure, return
+ * an iterator containing the values of the block. The returned iterator will be backed
+ * by the combination of the partially-unrolled block and the remaining elements of the
+ * original input iterator. The caller must either fully consume this iterator or call
+ * `close()` on it in order to free the storage memory consumed by the partially-unrolled
+ * block.
+ */
+ private[storage] def putIteratorAsValues[T](
+ blockId: BlockId,
+ values: Iterator[T],
+ classTag: ClassTag[T]): Either[PartiallyUnrolledIterator[T], Long] = {
+
+ val valuesHolder = new DeserializedValuesHolder[T](classTag)
+
+ putIterator(blockId, values, classTag, MemoryMode.ON_HEAP, valuesHolder) match {
+ case Right(storedSize) => Right(storedSize)
+ case Left(unrollMemoryUsedByThisBlock) =>
+ val unrolledIterator = if (valuesHolder.vector != null) {
+ valuesHolder.vector.iterator
+ } else {
+ valuesHolder.arrayValues.toIterator
+ }
+
Left(new PartiallyUnrolledIterator(
this,
MemoryMode.ON_HEAP,
unrollMemoryUsedByThisBlock,
- unrolled = arrayValues.toIterator,
- rest = Iterator.empty))
- }
- } else {
- // We ran out of space while unrolling the values for this block
- logUnrollFailureMessage(blockId, vector.estimateSize())
- Left(new PartiallyUnrolledIterator(
- this,
- MemoryMode.ON_HEAP,
- unrollMemoryUsedByThisBlock,
- unrolled = vector.iterator,
- rest = values))
+ unrolled = unrolledIterator,
+ rest = values))
}
}
/**
* Attempt to put the given block in memory store as bytes.
*
- * It's possible that the iterator is too large to materialize and store in memory. To avoid
- * OOM exceptions, this method will gradually unroll the iterator while periodically checking
- * whether there is enough free memory. If the block is successfully materialized, then the
- * temporary unroll memory used during the materialization is "transferred" to storage memory,
- * so we won't acquire more memory than is actually needed to store the block.
- *
* @return in case of success, the estimated size of the stored data. In case of failure,
* return a handle which allows the caller to either finish the serialization by
* spilling to disk or to deserialize the partially-serialized block and reconstruct
@@ -319,25 +331,8 @@ private[spark] class MemoryStore(
require(!contains(blockId), s"Block $blockId is already present in the MemoryStore")
- val allocator = memoryMode match {
- case MemoryMode.ON_HEAP => ByteBuffer.allocate _
- case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
- }
-
- // Whether there is still enough memory for us to continue unrolling this block
- var keepUnrolling = true
- // Number of elements unrolled so far
- var elementsUnrolled = 0L
- // How often to check whether we need to request more memory
- val memoryCheckPeriod = conf.get(UNROLL_MEMORY_CHECK_PERIOD)
- // Memory to request as a multiple of current bbos size
- val memoryGrowthFactor = conf.get(UNROLL_MEMORY_GROWTH_FACTOR)
// Initial per-task memory to request for unrolling blocks (bytes).
val initialMemoryThreshold = unrollMemoryThreshold
- // Keep track of unroll memory used by this particular block / putIterator() operation
- var unrollMemoryUsedByThisBlock = 0L
- // Underlying buffer for unrolling the block
- val redirectableStream = new RedirectableOutputStream
val chunkSize = if (initialMemoryThreshold > Int.MaxValue) {
logWarning(s"Initial memory threshold of ${Utils.bytesToString(initialMemoryThreshold)} " +
s"is too large to be set as chunk size. Chunk size has been capped to " +
@@ -346,85 +341,22 @@ private[spark] class MemoryStore(
} else {
initialMemoryThreshold.toInt
}
- val bbos = new ChunkedByteBufferOutputStream(chunkSize, allocator)
- redirectableStream.setOutputStream(bbos)
- val serializationStream: SerializationStream = {
- val autoPick = !blockId.isInstanceOf[StreamBlockId]
- val ser = serializerManager.getSerializer(classTag, autoPick).newInstance()
- ser.serializeStream(serializerManager.wrapForCompression(blockId, redirectableStream))
- }
- // Request enough memory to begin unrolling
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, memoryMode)
+ val valuesHolder = new SerializedValuesHolder[T](blockId, chunkSize, classTag,
+ memoryMode, serializerManager)
- if (!keepUnrolling) {
- logWarning(s"Failed to reserve initial memory threshold of " +
- s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.")
- } else {
- unrollMemoryUsedByThisBlock += initialMemoryThreshold
- }
-
- def reserveAdditionalMemoryIfNecessary(): Unit = {
- if (bbos.size > unrollMemoryUsedByThisBlock) {
- val amountToRequest = (bbos.size * memoryGrowthFactor - unrollMemoryUsedByThisBlock).toLong
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
- if (keepUnrolling) {
- unrollMemoryUsedByThisBlock += amountToRequest
- }
- }
- }
-
- // Unroll this block safely, checking whether we have exceeded our threshold
- while (values.hasNext && keepUnrolling) {
- serializationStream.writeObject(values.next())(classTag)
- elementsUnrolled += 1
- if (elementsUnrolled % memoryCheckPeriod == 0) {
- reserveAdditionalMemoryIfNecessary()
- }
- }
-
- // Make sure that we have enough memory to store the block. By this point, it is possible that
- // the block's actual memory usage has exceeded the unroll memory by a small amount, so we
- // perform one final call to attempt to allocate additional memory if necessary.
- if (keepUnrolling) {
- serializationStream.close()
- if (bbos.size > unrollMemoryUsedByThisBlock) {
- val amountToRequest = bbos.size - unrollMemoryUsedByThisBlock
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
- if (keepUnrolling) {
- unrollMemoryUsedByThisBlock += amountToRequest
- }
- }
- }
-
- if (keepUnrolling) {
- val entry = SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag)
- // Synchronize so that transfer is atomic
- memoryManager.synchronized {
- releaseUnrollMemoryForThisTask(memoryMode, unrollMemoryUsedByThisBlock)
- val success = memoryManager.acquireStorageMemory(blockId, entry.size, memoryMode)
- assert(success, "transferring unroll memory to storage memory failed")
- }
- entries.synchronized {
- entries.put(blockId, entry)
- }
- logInfo("Block %s stored as bytes in memory (estimated size %s, free %s)".format(
- blockId, Utils.bytesToString(entry.size),
- Utils.bytesToString(maxMemory - blocksMemoryUsed)))
- Right(entry.size)
- } else {
- // We ran out of space while unrolling the values for this block
- logUnrollFailureMessage(blockId, bbos.size)
- Left(
- new PartiallySerializedBlock(
+ putIterator(blockId, values, classTag, memoryMode, valuesHolder) match {
+ case Right(storedSize) => Right(storedSize)
+ case Left(unrollMemoryUsedByThisBlock) =>
+ Left(new PartiallySerializedBlock(
this,
serializerManager,
blockId,
- serializationStream,
- redirectableStream,
+ valuesHolder.serializationStream,
+ valuesHolder.redirectableStream,
unrollMemoryUsedByThisBlock,
memoryMode,
- bbos,
+ valuesHolder.bbos,
values,
classTag))
}
@@ -702,6 +634,94 @@ private[spark] class MemoryStore(
}
}
+private trait MemoryEntryBuilder[T] {
+ def preciseSize: Long
+ def build(): MemoryEntry[T]
+}
+
+private trait ValuesHolder[T] {
+ def storeValue(value: T): Unit
+ def estimatedSize(): Long
+
+ /**
+ * Note: After this method is called, the ValuesHolder is invalid, we can't store data and
+ * get estimate size again.
+ * @return a MemoryEntryBuilder which is used to build a memory entry and get the stored data
+ * size.
+ */
+ def getBuilder(): MemoryEntryBuilder[T]
+}
+
+/**
+ * A holder for storing the deserialized values.
+ */
+private class DeserializedValuesHolder[T] (classTag: ClassTag[T]) extends ValuesHolder[T] {
+ // Underlying vector for unrolling the block
+ var vector = new SizeTrackingVector[T]()(classTag)
+ var arrayValues: Array[T] = null
+
+ override def storeValue(value: T): Unit = {
+ vector += value
+ }
+
+ override def estimatedSize(): Long = {
+ vector.estimateSize()
+ }
+
+ override def getBuilder(): MemoryEntryBuilder[T] = new MemoryEntryBuilder[T] {
+ // We successfully unrolled the entirety of this block
+ arrayValues = vector.toArray
+ vector = null
+
+ override val preciseSize: Long = SizeEstimator.estimate(arrayValues)
+
+ override def build(): MemoryEntry[T] =
+ DeserializedMemoryEntry[T](arrayValues, preciseSize, classTag)
+ }
+}
+
+/**
+ * A holder for storing the serialized values.
+ */
+private class SerializedValuesHolder[T](
+ blockId: BlockId,
+ chunkSize: Int,
+ classTag: ClassTag[T],
+ memoryMode: MemoryMode,
+ serializerManager: SerializerManager) extends ValuesHolder[T] {
+ val allocator = memoryMode match {
+ case MemoryMode.ON_HEAP => ByteBuffer.allocate _
+ case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
+ }
+
+ val redirectableStream = new RedirectableOutputStream
+ val bbos = new ChunkedByteBufferOutputStream(chunkSize, allocator)
+ redirectableStream.setOutputStream(bbos)
+ val serializationStream: SerializationStream = {
+ val autoPick = !blockId.isInstanceOf[StreamBlockId]
+ val ser = serializerManager.getSerializer(classTag, autoPick).newInstance()
+ ser.serializeStream(serializerManager.wrapForCompression(blockId, redirectableStream))
+ }
+
+ override def storeValue(value: T): Unit = {
+ serializationStream.writeObject(value)(classTag)
+ }
+
+ override def estimatedSize(): Long = {
+ bbos.size
+ }
+
+ override def getBuilder(): MemoryEntryBuilder[T] = new MemoryEntryBuilder[T] {
+ // We successfully unrolled the entirety of this block
+ serializationStream.close()
+
+ override def preciseSize(): Long = bbos.size
+
+ override def build(): MemoryEntry[T] =
+ SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag)
+ }
+}
+
/**
* The result of a failed [[MemoryStore.putIteratorAsValues()]] call.
*
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 0adeb4058b6e..d6a025a6f12d 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -343,12 +343,14 @@ private[spark] object JettyUtils extends Logging {
-1,
connectionFactories: _*)
connector.setPort(port)
- connector.start()
+ connector.setHost(hostName)
+ connector.setReuseAddress(!Utils.isWindows)
// Currently we only use "SelectChannelConnector"
// Limit the max acceptor number to 8 so that we don't waste a lot of threads
connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8))
- connector.setHost(hostName)
+
+ connector.start()
// The number of selectors always equals to the number of acceptors
minThreads += connector.getAcceptors * 2
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index ba798df13c95..02cf19e00ecd 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -224,6 +224,7 @@ private[spark] object UIUtils extends Logging {
{commonHeaderNodes}
{if (showVisualization) vizHeaderNodes else Seq.empty}
{if (useDataTables) dataTablesHeaderNodes else Seq.empty}
+
{appName} - {title}
@@ -265,6 +266,7 @@ private[spark] object UIUtils extends Logging {
{commonHeaderNodes}
{if (useDataTables) dataTablesHeaderNodes else Seq.empty}
+
{title}
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
index 43adab7a35d6..902eb92b854f 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
@@ -48,10 +48,50 @@ private[ui] class EnvironmentPage(
classPathHeaders, classPathRow, appEnv.classpathEntries, fixedWidth = true)
val content =
-