|
- {worker.id}
+ {
+ if (worker.isAlive()) {
+
+ {worker.id}
+
+ } else {
+ worker.id
+ }
+ }
|
{worker.host}:{worker.port} |
{worker.state} |
@@ -247,10 +254,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
{driver.id} {killLink} |
{driver.submitDate} |
{driver.worker.map(w =>
-
- {w.id.toString}
- ).getOrElse("None")}
+ if (w.isAlive()) {
+
+ {w.id.toString}
+
+ } else {
+ w.id.toString
+ }).getOrElse("None")}
|
{driver.state} |
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 8b1c6bf2e5..0940f3c558 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -187,8 +187,7 @@ private[deploy] class Worker(
webUi = new WorkerWebUI(this, workDir, webUiPort)
webUi.bind()
- val scheme = if (webUi.sslOptions.enabled) "https" else "http"
- workerWebUiUrl = s"$scheme://$publicAddress:${webUi.boundPort}"
+ workerWebUiUrl = s"http://$publicAddress:${webUi.boundPort}"
registerWithMaster()
metricsSystem.registerSource(workerSource)
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 9501dd9cd8..3346f6dd1f 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -84,6 +84,16 @@ private[spark] class Executor(
// Start worker thread pool
private val threadPool = ThreadUtils.newDaemonCachedThreadPool("Executor task launch worker")
private val executorSource = new ExecutorSource(threadPool, executorId)
+ // Pool used for threads that supervise task killing / cancellation
+ private val taskReaperPool = ThreadUtils.newDaemonCachedThreadPool("Task reaper")
+ // For tasks which are in the process of being killed, this map holds the most recently created
+ // TaskReaper. All accesses to this map should be synchronized on the map itself (this isn't
+ // a ConcurrentHashMap because we use the synchronization for purposes other than simply guarding
+ // the integrity of the map's internal state). The purpose of this map is to prevent the creation
+ // of a separate TaskReaper for every killTask() of a given task. Instead, this map allows us to
+ // track whether an existing TaskReaper fulfills the role of a TaskReaper that we would otherwise
+ // create. The map key is a task id.
+ private val taskReaperForTask: HashMap[Long, TaskReaper] = HashMap[Long, TaskReaper]()
if (!isLocal) {
env.metricsSystem.registerSource(executorSource)
@@ -93,6 +103,9 @@ private[spark] class Executor(
// Whether to load classes in user jars before those in Spark jars
private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false)
+ // Whether to monitor killed / interrupted tasks
+ private val taskReaperEnabled = conf.getBoolean("spark.task.reaper.enabled", false)
+
// Create our ClassLoader
// do this after SparkEnv creation so can access the SecurityManager
private val urlClassLoader = createClassLoader()
@@ -148,9 +161,27 @@ private[spark] class Executor(
}
def killTask(taskId: Long, interruptThread: Boolean): Unit = {
- val tr = runningTasks.get(taskId)
- if (tr != null) {
- tr.kill(interruptThread)
+ val taskRunner = runningTasks.get(taskId)
+ if (taskRunner != null) {
+ if (taskReaperEnabled) {
+ val maybeNewTaskReaper: Option[TaskReaper] = taskReaperForTask.synchronized {
+ val shouldCreateReaper = taskReaperForTask.get(taskId) match {
+ case None => true
+ case Some(existingReaper) => interruptThread && !existingReaper.interruptThread
+ }
+ if (shouldCreateReaper) {
+ val taskReaper = new TaskReaper(taskRunner, interruptThread = interruptThread)
+ taskReaperForTask(taskId) = taskReaper
+ Some(taskReaper)
+ } else {
+ None
+ }
+ }
+ // Execute the TaskReaper from outside of the synchronized block.
+ maybeNewTaskReaper.foreach(taskReaperPool.execute)
+ } else {
+ taskRunner.kill(interruptThread = interruptThread)
+ }
}
}
@@ -161,12 +192,7 @@ private[spark] class Executor(
* @param interruptThread whether to interrupt the task thread
*/
def killAllTasks(interruptThread: Boolean) : Unit = {
- // kill all the running tasks
- for (taskRunner <- runningTasks.values().asScala) {
- if (taskRunner != null) {
- taskRunner.kill(interruptThread)
- }
- }
+ runningTasks.keys().asScala.foreach(t => killTask(t, interruptThread = interruptThread))
}
def stop(): Unit = {
@@ -192,13 +218,21 @@ private[spark] class Executor(
serializedTask: ByteBuffer)
extends Runnable {
+ val threadName = s"Executor task launch worker for task $taskId"
+
/** Whether this task has been killed. */
@volatile private var killed = false
+ @volatile private var threadId: Long = -1
+
+ def getThreadId: Long = threadId
+
/** Whether this task has been finished. */
@GuardedBy("TaskRunner.this")
private var finished = false
+ def isFinished: Boolean = synchronized { finished }
+
/** How much the JVM process has spent in GC when the task starts to run. */
@volatile var startGCTime: Long = _
@@ -229,9 +263,15 @@ private[spark] class Executor(
// ClosedByInterruptException during execBackend.statusUpdate which causes
// Executor to crash
Thread.interrupted()
+ // Notify any waiting TaskReapers. Generally there will only be one reaper per task but there
+ // is a rare corner-case where one task can have two reapers in case cancel(interrupt=False)
+ // is followed by cancel(interrupt=True). Thus we use notifyAll() to avoid a lost wakeup:
+ notifyAll()
}
override def run(): Unit = {
+ threadId = Thread.currentThread.getId
+ Thread.currentThread.setName(threadName)
val threadMXBean = ManagementFactory.getThreadMXBean
val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId)
val deserializeStartTime = System.currentTimeMillis()
@@ -431,6 +471,117 @@ private[spark] class Executor(
}
}
+ /**
+ * Supervises the killing / cancellation of a task by sending the interrupted flag, optionally
+ * sending a Thread.interrupt(), and monitoring the task until it finishes.
+ *
+ * Spark's current task cancellation / task killing mechanism is "best effort" because some tasks
+ * may not be interruptable or may not respond to their "killed" flags being set. If a significant
+ * fraction of a cluster's task slots are occupied by tasks that have been marked as killed but
+ * remain running then this can lead to a situation where new jobs and tasks are starved of
+ * resources that are being used by these zombie tasks.
+ *
+ * The TaskReaper was introduced in SPARK-18761 as a mechanism to monitor and clean up zombie
+ * tasks. For backwards-compatibility / backportability this component is disabled by default
+ * and must be explicitly enabled by setting `spark.task.reaper.enabled=true`.
+ *
+ * A TaskReaper is created for a particular task when that task is killed / cancelled. Typically
+ * a task will have only one TaskReaper, but it's possible for a task to have up to two reapers
+ * in case kill is called twice with different values for the `interrupt` parameter.
+ *
+ * Once created, a TaskReaper will run until its supervised task has finished running. If the
+ * TaskReaper has not been configured to kill the JVM after a timeout (i.e. if
+ * `spark.task.reaper.killTimeout < 0`) then this implies that the TaskReaper may run indefinitely
+ * if the supervised task never exits.
+ */
+ private class TaskReaper(
+ taskRunner: TaskRunner,
+ val interruptThread: Boolean)
+ extends Runnable {
+
+ private[this] val taskId: Long = taskRunner.taskId
+
+ private[this] val killPollingIntervalMs: Long =
+ conf.getTimeAsMs("spark.task.reaper.pollingInterval", "10s")
+
+ private[this] val killTimeoutMs: Long = conf.getTimeAsMs("spark.task.reaper.killTimeout", "-1")
+
+ private[this] val takeThreadDump: Boolean =
+ conf.getBoolean("spark.task.reaper.threadDump", true)
+
+ override def run(): Unit = {
+ val startTimeMs = System.currentTimeMillis()
+ def elapsedTimeMs = System.currentTimeMillis() - startTimeMs
+ def timeoutExceeded(): Boolean = killTimeoutMs > 0 && elapsedTimeMs > killTimeoutMs
+ try {
+ // Only attempt to kill the task once. If interruptThread = false then a second kill
+ // attempt would be a no-op and if interruptThread = true then it may not be safe or
+ // effective to interrupt multiple times:
+ taskRunner.kill(interruptThread = interruptThread)
+ // Monitor the killed task until it exits. The synchronization logic here is complicated
+ // because we don't want to synchronize on the taskRunner while possibly taking a thread
+ // dump, but we also need to be careful to avoid races between checking whether the task
+ // has finished and wait()ing for it to finish.
+ var finished: Boolean = false
+ while (!finished && !timeoutExceeded()) {
+ taskRunner.synchronized {
+ // We need to synchronize on the TaskRunner while checking whether the task has
+ // finished in order to avoid a race where the task is marked as finished right after
+ // we check and before we call wait().
+ if (taskRunner.isFinished) {
+ finished = true
+ } else {
+ taskRunner.wait(killPollingIntervalMs)
+ }
+ }
+ if (taskRunner.isFinished) {
+ finished = true
+ } else {
+ logWarning(s"Killed task $taskId is still running after $elapsedTimeMs ms")
+ if (takeThreadDump) {
+ try {
+ Utils.getThreadDumpForThread(taskRunner.getThreadId).foreach { thread =>
+ if (thread.threadName == taskRunner.threadName) {
+ logWarning(s"Thread dump from task $taskId:\n${thread.stackTrace}")
+ }
+ }
+ } catch {
+ case NonFatal(e) =>
+ logWarning("Exception thrown while obtaining thread dump: ", e)
+ }
+ }
+ }
+ }
+
+ if (!taskRunner.isFinished && timeoutExceeded()) {
+ if (isLocal) {
+ logError(s"Killed task $taskId could not be stopped within $killTimeoutMs ms; " +
+ "not killing JVM because we are running in local mode.")
+ } else {
+ // In non-local-mode, the exception thrown here will bubble up to the uncaught exception
+ // handler and cause the executor JVM to exit.
+ throw new SparkException(
+ s"Killing executor JVM because killed task $taskId could not be stopped within " +
+ s"$killTimeoutMs ms.")
+ }
+ }
+ } finally {
+ // Clean up entries in the taskReaperForTask map.
+ taskReaperForTask.synchronized {
+ taskReaperForTask.get(taskId).foreach { taskReaperInMap =>
+ if (taskReaperInMap eq this) {
+ taskReaperForTask.remove(taskId)
+ } else {
+ // This must have been a TaskReaper where interruptThread == false where a subsequent
+ // killTask() call for the same task had interruptThread == true and overwrote the
+ // map entry.
+ }
+ }
+ }
+ }
+ }
+ }
+
/**
* Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes
* created by the interpreter to the search path
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 a69a2b5645..aba429bcdc 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
@@ -114,11 +114,21 @@ package object config {
.intConf
.createWithDefault(2)
+ private[spark] val MAX_FAILURES_PER_EXEC =
+ ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor")
+ .intConf
+ .createWithDefault(2)
+
private[spark] val MAX_FAILURES_PER_EXEC_STAGE =
ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor")
.intConf
.createWithDefault(2)
+ private[spark] val MAX_FAILED_EXEC_PER_NODE =
+ ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode")
+ .intConf
+ .createWithDefault(2)
+
private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE =
ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode")
.intConf
@@ -198,12 +208,13 @@ package object config {
.createWithDefault(0)
private[spark] val DRIVER_BLOCK_MANAGER_PORT = ConfigBuilder("spark.driver.blockManager.port")
- .doc("Port to use for the block managed on the driver.")
+ .doc("Port to use for the block manager on the driver.")
.fallbackConf(BLOCK_MANAGER_PORT)
private[spark] val IGNORE_CORRUPT_FILES = ConfigBuilder("spark.files.ignoreCorruptFiles")
.doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " +
- "encountering corrupt files and contents that have been read will still be returned.")
+ "encountering corrupted or non-existing files and contents that have been read will still " +
+ "be returned.")
.booleanConf
.createWithDefault(false)
diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
index aaeb3d0038..6de1fc0685 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
@@ -146,7 +146,7 @@ object SparkHadoopMapReduceWriter extends Logging {
case c: Configurable => c.setConf(hadoopConf)
case _ => ()
}
- val writer = taskFormat.getRecordWriter(taskContext)
+ var writer = taskFormat.getRecordWriter(taskContext)
.asInstanceOf[RecordWriter[K, V]]
require(writer != null, "Unable to obtain RecordWriter")
var recordsWritten = 0L
@@ -154,6 +154,7 @@ object SparkHadoopMapReduceWriter extends Logging {
// Write all rows in RDD partition.
try {
val ret = Utils.tryWithSafeFinallyAndFailureCallbacks {
+ // Write rows out, release resource and commit the task.
while (iterator.hasNext) {
val pair = iterator.next()
writer.write(pair._1, pair._2)
@@ -163,12 +164,23 @@ object SparkHadoopMapReduceWriter extends Logging {
outputMetricsAndBytesWrittenCallback, recordsWritten)
recordsWritten += 1
}
-
+ if (writer != null) {
+ writer.close(taskContext)
+ writer = null
+ }
committer.commitTask(taskContext)
}(catchBlock = {
- committer.abortTask(taskContext)
- logError(s"Task ${taskContext.getTaskAttemptID} aborted.")
- }, finallyBlock = writer.close(taskContext))
+ // If there is an error, release resource and then abort the task.
+ try {
+ if (writer != null) {
+ writer.close(taskContext)
+ writer = null
+ }
+ } finally {
+ committer.abortTask(taskContext)
+ logError(s"Task ${taskContext.getTaskAttemptID} aborted.")
+ }
+ })
outputMetricsAndBytesWrittenCallback.foreach {
case (om, callback) =>
diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala
index 86874e2067..df520f804b 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala
@@ -17,6 +17,8 @@
package org.apache.spark.network.netty
+import scala.collection.JavaConverters._
+
import org.apache.spark.SparkConf
import org.apache.spark.network.util.{ConfigProvider, TransportConf}
@@ -58,6 +60,10 @@ object SparkTransportConf {
new TransportConf(module, new ConfigProvider {
override def get(name: String): String = conf.get(name)
+
+ override def getAll(): java.lang.Iterable[java.util.Map.Entry[String, String]] = {
+ conf.getAll.toMap.asJava.entrySet()
+ }
})
}
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 ae4320d458..a83e139c13 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -131,9 +131,9 @@ class HadoopRDD[K, V](
minPartitions)
}
- protected val jobConfCacheKey = "rdd_%d_job_conf".format(id)
+ protected val jobConfCacheKey: String = "rdd_%d_job_conf".format(id)
- protected val inputFormatCacheKey = "rdd_%d_input_format".format(id)
+ protected val inputFormatCacheKey: String = "rdd_%d_input_format".format(id)
// used to build JobTracker ID
private val createTime = new Date()
@@ -210,22 +210,24 @@ class HadoopRDD[K, V](
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
val iter = new NextIterator[(K, V)] {
- val split = theSplit.asInstanceOf[HadoopPartition]
+ private val split = theSplit.asInstanceOf[HadoopPartition]
logInfo("Input split: " + split.inputSplit)
- val jobConf = getJobConf()
+ private val jobConf = getJobConf()
- val inputMetrics = context.taskMetrics().inputMetrics
- val existingBytesRead = inputMetrics.bytesRead
+ private val inputMetrics = context.taskMetrics().inputMetrics
+ private val existingBytesRead = inputMetrics.bytesRead
- // Sets the thread local variable for the file's name
+ // Sets InputFileBlockHolder for the file block's information
split.inputSplit.value match {
- case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString)
- case _ => InputFileNameHolder.unsetInputFileName()
+ case fs: FileSplit =>
+ InputFileBlockHolder.set(fs.getPath.toString, fs.getStart, fs.getLength)
+ case _ =>
+ InputFileBlockHolder.unset()
}
// Find a function that will return the FileSystem bytes read by this thread. Do this before
// creating RecordReader, because RecordReader's constructor might read some bytes
- val getBytesReadCallback: Option[() => Long] = split.inputSplit.value match {
+ private val getBytesReadCallback: Option[() => Long] = split.inputSplit.value match {
case _: FileSplit | _: CombineFileSplit =>
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback()
case _ => None
@@ -235,29 +237,39 @@ class HadoopRDD[K, V](
// If we do a coalesce, however, we are likely to compute multiple partitions in the same
// task and in the same thread, in which case we need to avoid override values written by
// previous partitions (SPARK-13071).
- def updateBytesRead(): Unit = {
+ private def updateBytesRead(): Unit = {
getBytesReadCallback.foreach { getBytesRead =>
inputMetrics.setBytesRead(existingBytesRead + getBytesRead())
}
}
- var reader: RecordReader[K, V] = null
- val inputFormat = getInputFormat(jobConf)
+ private var reader: RecordReader[K, V] = null
+ private val inputFormat = getInputFormat(jobConf)
HadoopRDD.addLocalConfiguration(
new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(createTime),
context.stageId, theSplit.index, context.attemptNumber, jobConf)
- reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
+ reader =
+ try {
+ inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
+ } catch {
+ case e: IOException if ignoreCorruptFiles =>
+ logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e)
+ finished = true
+ null
+ }
// Register an on-task-completion callback to close the input stream.
context.addTaskCompletionListener{ context => closeIfNeeded() }
- val key: K = reader.createKey()
- val value: V = reader.createValue()
+ private val key: K = if (reader == null) null.asInstanceOf[K] else reader.createKey()
+ private val value: V = if (reader == null) null.asInstanceOf[V] else reader.createValue()
override def getNext(): (K, V) = {
try {
finished = !reader.next(key, value)
} catch {
- case e: IOException if ignoreCorruptFiles => finished = true
+ case e: IOException if ignoreCorruptFiles =>
+ logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e)
+ finished = true
}
if (!finished) {
inputMetrics.incRecordsRead(1)
@@ -270,7 +282,7 @@ class HadoopRDD[K, V](
override def close() {
if (reader != null) {
- InputFileNameHolder.unsetInputFileName()
+ InputFileBlockHolder.unset()
// Close the reader and release it. Note: it's very important that we don't close the
// reader more than once, since that exposes us to MAPREDUCE-5918 when running against
// Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic
diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala
new file mode 100644
index 0000000000..9ba476d2ba
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.rdd
+
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * This holds file names of the current Spark task. This is used in HadoopRDD,
+ * FileScanRDD, NewHadoopRDD and InputFileName function in Spark SQL.
+ */
+private[spark] object InputFileBlockHolder {
+ /**
+ * A wrapper around some input file information.
+ *
+ * @param filePath path of the file read, or empty string if not available.
+ * @param startOffset starting offset, in bytes, or -1 if not available.
+ * @param length size of the block, in bytes, or -1 if not available.
+ */
+ private class FileBlock(val filePath: UTF8String, val startOffset: Long, val length: Long) {
+ def this() {
+ this(UTF8String.fromString(""), -1, -1)
+ }
+ }
+
+ /**
+ * The thread variable for the name of the current file being read. This is used by
+ * the InputFileName function in Spark SQL.
+ */
+ private[this] val inputBlock: ThreadLocal[FileBlock] = new ThreadLocal[FileBlock] {
+ override protected def initialValue(): FileBlock = new FileBlock
+ }
+
+ /**
+ * Returns the holding file name or empty string if it is unknown.
+ */
+ def getInputFilePath: UTF8String = inputBlock.get().filePath
+
+ /**
+ * Returns the starting offset of the block currently being read, or -1 if it is unknown.
+ */
+ def getStartOffset: Long = inputBlock.get().startOffset
+
+ /**
+ * Returns the length of the block being read, or -1 if it is unknown.
+ */
+ def getLength: Long = inputBlock.get().length
+
+ /**
+ * Sets the thread-local input block.
+ */
+ def set(filePath: String, startOffset: Long, length: Long): Unit = {
+ require(filePath != null, "filePath cannot be null")
+ require(startOffset >= 0, s"startOffset ($startOffset) cannot be negative")
+ require(length >= 0, s"length ($length) cannot be negative")
+ inputBlock.set(new FileBlock(UTF8String.fromString(filePath), startOffset, length))
+ }
+
+ /**
+ * Clears the input file block to default value.
+ */
+ def unset(): Unit = inputBlock.remove()
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala
deleted file mode 100644
index 960c91a154..0000000000
--- a/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.rdd
-
-import org.apache.spark.unsafe.types.UTF8String
-
-/**
- * This holds file names of the current Spark task. This is used in HadoopRDD,
- * FileScanRDD, NewHadoopRDD and InputFileName function in Spark SQL.
- *
- * The returned value should never be null but empty string if it is unknown.
- */
-private[spark] object InputFileNameHolder {
- /**
- * The thread variable for the name of the current file being read. This is used by
- * the InputFileName function in Spark SQL.
- */
- private[this] val inputFileName: ThreadLocal[UTF8String] = new ThreadLocal[UTF8String] {
- override protected def initialValue(): UTF8String = UTF8String.fromString("")
- }
-
- /**
- * Returns the holding file name or empty string if it is unknown.
- */
- def getInputFileName(): UTF8String = inputFileName.get()
-
- private[spark] def setInputFileName(file: String) = {
- require(file != null, "The input file name cannot be null")
- inputFileName.set(UTF8String.fromString(file))
- }
-
- private[spark] def unsetInputFileName(): Unit = inputFileName.remove()
-
-}
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 c783e13752..733e85f305 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -132,61 +132,79 @@ class NewHadoopRDD[K, V](
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
val iter = new Iterator[(K, V)] {
- val split = theSplit.asInstanceOf[NewHadoopPartition]
+ private val split = theSplit.asInstanceOf[NewHadoopPartition]
logInfo("Input split: " + split.serializableHadoopSplit)
- val conf = getConf
+ private val conf = getConf
- val inputMetrics = context.taskMetrics().inputMetrics
- val existingBytesRead = inputMetrics.bytesRead
+ private val inputMetrics = context.taskMetrics().inputMetrics
+ private val existingBytesRead = inputMetrics.bytesRead
- // Sets the thread local variable for the file's name
+ // Sets InputFileBlockHolder for the file block's information
split.serializableHadoopSplit.value match {
- case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString)
- case _ => InputFileNameHolder.unsetInputFileName()
+ case fs: FileSplit =>
+ InputFileBlockHolder.set(fs.getPath.toString, fs.getStart, fs.getLength)
+ case _ =>
+ InputFileBlockHolder.unset()
}
// Find a function that will return the FileSystem bytes read by this thread. Do this before
// creating RecordReader, because RecordReader's constructor might read some bytes
- val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match {
- case _: FileSplit | _: CombineFileSplit =>
- SparkHadoopUtil.get.getFSBytesReadOnThreadCallback()
- case _ => None
- }
+ private val getBytesReadCallback: Option[() => Long] =
+ split.serializableHadoopSplit.value match {
+ case _: FileSplit | _: CombineFileSplit =>
+ SparkHadoopUtil.get.getFSBytesReadOnThreadCallback()
+ case _ => None
+ }
// For Hadoop 2.5+, we get our input bytes from thread-local Hadoop FileSystem statistics.
// If we do a coalesce, however, we are likely to compute multiple partitions in the same
// task and in the same thread, in which case we need to avoid override values written by
// previous partitions (SPARK-13071).
- def updateBytesRead(): Unit = {
+ private def updateBytesRead(): Unit = {
getBytesReadCallback.foreach { getBytesRead =>
inputMetrics.setBytesRead(existingBytesRead + getBytesRead())
}
}
- val format = inputFormatClass.newInstance
+ private val format = inputFormatClass.newInstance
format match {
case configurable: Configurable =>
configurable.setConf(conf)
case _ =>
}
- val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
- val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
- private var reader = format.createRecordReader(
- split.serializableHadoopSplit.value, hadoopAttemptContext)
- reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
+ private val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
+ private val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+ private var finished = false
+ private var reader =
+ try {
+ val _reader = format.createRecordReader(
+ split.serializableHadoopSplit.value, hadoopAttemptContext)
+ _reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
+ _reader
+ } catch {
+ case e: IOException if ignoreCorruptFiles =>
+ logWarning(
+ s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}",
+ e)
+ finished = true
+ null
+ }
// Register an on-task-completion callback to close the input stream.
context.addTaskCompletionListener(context => close())
- var havePair = false
- var finished = false
- var recordsSinceMetricsUpdate = 0
+ private var havePair = false
+ private var recordsSinceMetricsUpdate = 0
override def hasNext: Boolean = {
if (!finished && !havePair) {
try {
finished = !reader.nextKeyValue
} catch {
- case e: IOException if ignoreCorruptFiles => finished = true
+ case e: IOException if ignoreCorruptFiles =>
+ logWarning(
+ s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}",
+ e)
+ finished = true
}
if (finished) {
// Close and release the reader here; close() will also be called when the task
@@ -215,7 +233,7 @@ class NewHadoopRDD[K, V](
private def close() {
if (reader != null) {
- InputFileNameHolder.unsetInputFileName()
+ InputFileBlockHolder.unset()
// Close the reader and release it. Note: it's very important that we don't close the
// reader more than once, since that exposes us to MAPREDUCE-5918 when running against
// Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic
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 d285e917b8..374abccf6a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1746,7 +1746,7 @@ abstract class RDD[T: ClassTag](
/**
* Clears the dependencies of this RDD. This method must ensure that all references
- * to the original parent RDDs is removed to enable the parent RDDs to be garbage
+ * to the original parent RDDs are removed to enable the parent RDDs to be garbage
* collected. Subclasses of RDD may override this method for implementing their own cleaning
* logic. See [[org.apache.spark.rdd.UnionRDD]] for an example.
*/
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index 29d5d74650..26eaa9aa3d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -25,10 +25,6 @@ import org.apache.spark.serializer.Serializer
private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
override val index: Int = idx
-
- override def hashCode(): Int = index
-
- override def equals(other: Any): Boolean = super.equals(other)
}
/**
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala
index f527ec86ab..117f51c5b8 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala
@@ -18,7 +18,7 @@
package org.apache.spark.rpc
/**
- * A callback that [[RpcEndpoint]] can use it to send back a message or failure. It's thread-safe
+ * A callback that [[RpcEndpoint]] can use to send back a message or failure. It's thread-safe
* and can be called in any thread.
*/
private[spark] trait RpcCallContext {
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
index bbc4163814..530743c036 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
@@ -146,11 +146,6 @@ private[spark] abstract class RpcEnv(conf: SparkConf) {
* @param uri URI with location of the file.
*/
def openChannel(uri: String): ReadableByteChannel
-
- /**
- * Return if the current thread is a RPC thread.
- */
- def isInRPCThread: Boolean
}
/**
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
index 2761d39e37..efd26486ab 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
@@ -24,7 +24,7 @@ import scala.concurrent.duration._
import scala.util.control.NonFatal
import org.apache.spark.{SparkConf, SparkException}
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{ThreadUtils, Utils}
/**
* An exception thrown if RpcTimeout modifies a [[TimeoutException]].
@@ -72,15 +72,9 @@ private[spark] class RpcTimeout(val duration: FiniteDuration, val timeoutProp: S
* is still not ready
*/
def awaitResult[T](future: Future[T]): T = {
- val wrapAndRethrow: PartialFunction[Throwable, T] = {
- case NonFatal(t) =>
- throw new SparkException("Exception thrown in awaitResult", t)
- }
try {
- // scalastyle:off awaitresult
- Await.result(future, duration)
- // scalastyle:on awaitresult
- } catch addMessageIfTimeout.orElse(wrapAndRethrow)
+ ThreadUtils.awaitResult(future, duration)
+ } catch addMessageIfTimeout
}
}
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
index 67baabd2cb..a02cf30a5d 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
@@ -201,7 +201,6 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging {
/** Message loop used for dispatching messages. */
private class MessageLoop extends Runnable {
override def run(): Unit = {
- NettyRpcEnv.rpcThreadFlag.value = true
try {
while (true) {
try {
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 0b8cd144a2..e56943da13 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
@@ -407,14 +407,9 @@ private[netty] class NettyRpcEnv(
}
}
-
- override def isInRPCThread: Boolean = NettyRpcEnv.rpcThreadFlag.value
}
private[netty] object NettyRpcEnv extends Logging {
-
- private[netty] val rpcThreadFlag = new DynamicVariable[Boolean](false)
-
/**
* When deserializing the [[NettyRpcEndpointRef]], it needs a reference to [[NettyRpcEnv]].
* Use `currentEnv` to wrap the deserialization codes. E.g.,
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 fca4c6d37e..bf7a62ea33 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
@@ -17,10 +17,274 @@
package org.apache.spark.scheduler
+import java.util.concurrent.atomic.AtomicReference
+
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{Clock, SystemClock, Utils}
+
+/**
+ * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting
+ * executors and nodes across an entire application (with a periodic expiry). TaskSetManagers add
+ * additional blacklisting of executors and nodes for individual tasks and stages which works in
+ * concert with the blacklisting here.
+ *
+ * The tracker needs to deal with a variety of workloads, eg.:
+ *
+ * * bad user code -- this may lead to many task failures, but that should not count against
+ * individual executors
+ * * many small stages -- this may prevent a bad executor for having many failures within one
+ * stage, but still many failures over the entire application
+ * * "flaky" executors -- they don't fail every task, but are still faulty enough to merit
+ * blacklisting
+ *
+ * See the design doc on SPARK-8425 for a more in-depth discussion.
+ *
+ * THREADING: As with most helpers of TaskSchedulerImpl, this is not thread-safe. Though it is
+ * called by multiple threads, callers must already have a lock on the TaskSchedulerImpl. The
+ * one exception is [[nodeBlacklist()]], which can be called without holding a lock.
+ */
+private[scheduler] class BlacklistTracker (
+ conf: SparkConf,
+ clock: Clock = new SystemClock()) extends Logging {
+
+ BlacklistTracker.validateBlacklistConfs(conf)
+ private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC)
+ private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE)
+ val BLACKLIST_TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf)
+
+ /**
+ * A map from executorId to information on task failures. Tracks the time of each task failure,
+ * so that we can avoid blacklisting executors due to failures that are very far apart. We do not
+ * actively remove from this as soon as tasks hit their timeouts, to avoid the time it would take
+ * to do so. But it will not grow too large, because as soon as an executor gets too many
+ * failures, we blacklist the executor and remove its entry here.
+ */
+ private val executorIdToFailureList = new HashMap[String, ExecutorFailureList]()
+ val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
+ val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
+ /**
+ * An immutable copy of the set of nodes that are currently blacklisted. Kept in an
+ * AtomicReference to make [[nodeBlacklist()]] thread-safe.
+ */
+ private val _nodeBlacklist = new AtomicReference[Set[String]](Set())
+ /**
+ * Time when the next blacklist will expire. Used as a
+ * shortcut to avoid iterating over all entries in the blacklist when none will have expired.
+ */
+ var nextExpiryTime: Long = Long.MaxValue
+ /**
+ * Mapping from nodes to all of the executors that have been blacklisted on that node. We do *not*
+ * remove from this when executors are removed from spark, so we can track when we get multiple
+ * successive blacklisted executors on one node. Nonetheless, it will not grow too large because
+ * there cannot be many blacklisted executors on one node, before we stop requesting more
+ * executors on that node, and we clean up the list of blacklisted executors once an executor has
+ * been blacklisted for BLACKLIST_TIMEOUT_MILLIS.
+ */
+ val nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]()
+
+ /**
+ * Un-blacklists executors and nodes that have been blacklisted for at least
+ * BLACKLIST_TIMEOUT_MILLIS
+ */
+ def applyBlacklistTimeout(): Unit = {
+ val now = clock.getTimeMillis()
+ // quickly check if we've got anything to expire from blacklist -- if not, avoid doing any work
+ if (now > nextExpiryTime) {
+ // Apply the timeout to blacklisted nodes and executors
+ val execsToUnblacklist = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys
+ if (execsToUnblacklist.nonEmpty) {
+ // Un-blacklist any executors that have been blacklisted longer than the blacklist timeout.
+ logInfo(s"Removing executors $execsToUnblacklist from blacklist because the blacklist " +
+ s"for those executors has timed out")
+ execsToUnblacklist.foreach { exec =>
+ val status = executorIdToBlacklistStatus.remove(exec).get
+ val failedExecsOnNode = nodeToBlacklistedExecs(status.node)
+ failedExecsOnNode.remove(exec)
+ if (failedExecsOnNode.isEmpty) {
+ nodeToBlacklistedExecs.remove(status.node)
+ }
+ }
+ }
+ val nodesToUnblacklist = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys
+ if (nodesToUnblacklist.nonEmpty) {
+ // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout.
+ logInfo(s"Removing nodes $nodesToUnblacklist from blacklist because the blacklist " +
+ s"has timed out")
+ nodeIdToBlacklistExpiryTime --= nodesToUnblacklist
+ _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet)
+ }
+ updateNextExpiryTime()
+ }
+ }
+
+ private def updateNextExpiryTime(): Unit = {
+ val execMinExpiry = if (executorIdToBlacklistStatus.nonEmpty) {
+ executorIdToBlacklistStatus.map{_._2.expiryTime}.min
+ } else {
+ Long.MaxValue
+ }
+ val nodeMinExpiry = if (nodeIdToBlacklistExpiryTime.nonEmpty) {
+ nodeIdToBlacklistExpiryTime.values.min
+ } else {
+ Long.MaxValue
+ }
+ nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
+ }
+
+
+ def updateBlacklistForSuccessfulTaskSet(
+ stageId: Int,
+ stageAttemptId: Int,
+ failuresByExec: HashMap[String, ExecutorFailuresInTaskSet]): Unit = {
+ // if any tasks failed, we count them towards the overall failure count for the executor at
+ // this point.
+ val now = clock.getTimeMillis()
+ failuresByExec.foreach { case (exec, failuresInTaskSet) =>
+ val appFailuresOnExecutor =
+ executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList)
+ appFailuresOnExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet)
+ appFailuresOnExecutor.dropFailuresWithTimeoutBefore(now)
+ val newTotal = appFailuresOnExecutor.numUniqueTaskFailures
+
+ val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS
+ // If this pushes the total number of failures over the threshold, blacklist the executor.
+ // If its already blacklisted, we avoid "re-blacklisting" (which can happen if there were
+ // other tasks already running in another taskset when it got blacklisted), because it makes
+ // some of the logic around expiry times a little more confusing. But it also wouldn't be a
+ // problem to re-blacklist, with a later expiry time.
+ if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToBlacklistStatus.contains(exec)) {
+ logInfo(s"Blacklisting executor id: $exec because it has $newTotal" +
+ s" task failures in successful task sets")
+ val node = failuresInTaskSet.node
+ executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists))
+ updateNextExpiryTime()
+
+ // In addition to blacklisting the executor, we also update the data for failures on the
+ // node, and potentially put the entire node into a blacklist as well.
+ val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(node, HashSet[String]())
+ blacklistedExecsOnNode += exec
+ // If the node is already in the blacklist, we avoid adding it again with a later expiry
+ // time.
+ if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE &&
+ !nodeIdToBlacklistExpiryTime.contains(node)) {
+ logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " +
+ s"executors blacklisted: ${blacklistedExecsOnNode}")
+ nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists)
+ _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet)
+ }
+ }
+ }
+ }
+
+ def isExecutorBlacklisted(executorId: String): Boolean = {
+ executorIdToBlacklistStatus.contains(executorId)
+ }
+
+ /**
+ * Get the full set of nodes that are blacklisted. Unlike other methods in this class, this *IS*
+ * thread-safe -- no lock required on a taskScheduler.
+ */
+ def nodeBlacklist(): Set[String] = {
+ _nodeBlacklist.get()
+ }
+
+ def isNodeBlacklisted(node: String): Boolean = {
+ nodeIdToBlacklistExpiryTime.contains(node)
+ }
+
+ def handleRemovedExecutor(executorId: String): Unit = {
+ // We intentionally do not clean up executors that are already blacklisted in
+ // nodeToBlacklistedExecs, so that if another executor on the same node gets blacklisted, we can
+ // blacklist the entire node. We also can't clean up executorIdToBlacklistStatus, so we can
+ // eventually remove the executor after the timeout. Despite not clearing those structures
+ // here, we don't expect they will grow too big since you won't get too many executors on one
+ // node, and the timeout will clear it up periodically in any case.
+ executorIdToFailureList -= executorId
+ }
+
+
+ /**
+ * Tracks all failures for one executor (that have not passed the timeout).
+ *
+ * In general we actually expect this to be extremely small, since it won't contain more than the
+ * maximum number of task failures before an executor is failed (default 2).
+ */
+ private[scheduler] final class ExecutorFailureList extends Logging {
+
+ private case class TaskId(stage: Int, stageAttempt: Int, taskIndex: Int)
+
+ /**
+ * All failures on this executor in successful task sets.
+ */
+ private var failuresAndExpiryTimes = ArrayBuffer[(TaskId, Long)]()
+ /**
+ * As an optimization, we track the min expiry time over all entries in failuresAndExpiryTimes
+ * so its quick to tell if there are any failures with expiry before the current time.
+ */
+ private var minExpiryTime = Long.MaxValue
+
+ def addFailures(
+ stage: Int,
+ stageAttempt: Int,
+ failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = {
+ failuresInTaskSet.taskToFailureCountAndFailureTime.foreach {
+ case (taskIdx, (_, failureTime)) =>
+ val expiryTime = failureTime + BLACKLIST_TIMEOUT_MILLIS
+ failuresAndExpiryTimes += ((TaskId(stage, stageAttempt, taskIdx), expiryTime))
+ if (expiryTime < minExpiryTime) {
+ minExpiryTime = expiryTime
+ }
+ }
+ }
+
+ /**
+ * The number of unique tasks that failed on this executor. Only counts failures within the
+ * timeout, and in successful tasksets.
+ */
+ def numUniqueTaskFailures: Int = failuresAndExpiryTimes.size
+
+ def isEmpty: Boolean = failuresAndExpiryTimes.isEmpty
+
+ /**
+ * Apply the timeout to individual tasks. This is to prevent one-off failures that are very
+ * spread out in time (and likely have nothing to do with problems on the executor) from
+ * triggering blacklisting. However, note that we do *not* remove executors and nodes from
+ * the blacklist as we expire individual task failures -- each have their own timeout. Eg.,
+ * suppose:
+ * * timeout = 10, maxFailuresPerExec = 2
+ * * Task 1 fails on exec 1 at time 0
+ * * Task 2 fails on exec 1 at time 5
+ * --> exec 1 is blacklisted from time 5 - 15.
+ * This is to simplify the implementation, as well as keep the behavior easier to understand
+ * for the end user.
+ */
+ def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = {
+ if (minExpiryTime < dropBefore) {
+ var newMinExpiry = Long.MaxValue
+ val newFailures = new ArrayBuffer[(TaskId, Long)]
+ failuresAndExpiryTimes.foreach { case (task, expiryTime) =>
+ if (expiryTime >= dropBefore) {
+ newFailures += ((task, expiryTime))
+ if (expiryTime < newMinExpiry) {
+ newMinExpiry = expiryTime
+ }
+ }
+ }
+ failuresAndExpiryTimes = newFailures
+ minExpiryTime = newMinExpiry
+ }
+ }
+
+ override def toString(): String = {
+ s"failures = $failuresAndExpiryTimes"
+ }
+ }
+
+}
private[scheduler] object BlacklistTracker extends Logging {
@@ -80,7 +344,9 @@ private[scheduler] object BlacklistTracker extends Logging {
config.MAX_TASK_ATTEMPTS_PER_EXECUTOR,
config.MAX_TASK_ATTEMPTS_PER_NODE,
config.MAX_FAILURES_PER_EXEC_STAGE,
- config.MAX_FAILED_EXEC_PER_NODE_STAGE
+ config.MAX_FAILED_EXEC_PER_NODE_STAGE,
+ config.MAX_FAILURES_PER_EXEC,
+ config.MAX_FAILED_EXEC_PER_NODE
).foreach { config =>
val v = conf.get(config)
if (v <= 0) {
@@ -112,3 +378,5 @@ private[scheduler] object BlacklistTracker extends Logging {
}
}
}
+
+private final case class BlacklistedExecutor(node: String, expiryTime: Long)
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 7fde34d897..6177bafc11 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1009,13 +1009,14 @@ class DAGScheduler(
}
val tasks: Seq[Task[_]] = try {
+ val serializedTaskMetrics = closureSerializer.serialize(stage.latestInfo.taskMetrics).array()
stage match {
case stage: ShuffleMapStage =>
partitionsToCompute.map { id =>
val locs = taskIdToLocations(id)
val part = stage.rdd.partitions(id)
new ShuffleMapTask(stage.id, stage.latestInfo.attemptId,
- taskBinary, part, locs, stage.latestInfo.taskMetrics, properties, Option(jobId),
+ taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId),
Option(sc.applicationId), sc.applicationAttemptId)
}
@@ -1025,7 +1026,7 @@ class DAGScheduler(
val part = stage.rdd.partitions(p)
val locs = taskIdToLocations(id)
new ResultTask(stage.id, stage.latestInfo.attemptId,
- taskBinary, part, locs, id, properties, stage.latestInfo.taskMetrics,
+ taskBinary, part, locs, id, properties, serializedTaskMetrics,
Option(jobId), Option(sc.applicationId), sc.applicationAttemptId)
}
}
@@ -1256,27 +1257,46 @@ class DAGScheduler(
s"longer running")
}
- if (disallowStageRetryForTest) {
- abortStage(failedStage, "Fetch failure will not retry stage due to testing config",
- None)
- } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) {
- abortStage(failedStage, s"$failedStage (${failedStage.name}) " +
- s"has failed the maximum allowable number of " +
- s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " +
- s"Most recent failure reason: ${failureMessage}", None)
- } else {
- if (failedStages.isEmpty) {
- // Don't schedule an event to resubmit failed stages if failed isn't empty, because
- // in that case the event will already have been scheduled.
- // TODO: Cancel running tasks in the stage
- logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " +
- s"$failedStage (${failedStage.name}) due to fetch failure")
- messageScheduler.schedule(new Runnable {
- override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages)
- }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS)
+ val shouldAbortStage =
+ failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId) ||
+ disallowStageRetryForTest
+
+ if (shouldAbortStage) {
+ val abortMessage = if (disallowStageRetryForTest) {
+ "Fetch failure will not retry stage due to testing config"
+ } else {
+ s"""$failedStage (${failedStage.name})
+ |has failed the maximum allowable number of
+ |times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}.
+ |Most recent failure reason: $failureMessage""".stripMargin.replaceAll("\n", " ")
}
+ abortStage(failedStage, abortMessage, None)
+ } else { // update failedStages and make sure a ResubmitFailedStages event is enqueued
+ // TODO: Cancel running tasks in the failed stage -- cf. SPARK-17064
+ val noResubmitEnqueued = !failedStages.contains(failedStage)
failedStages += failedStage
failedStages += mapStage
+ if (noResubmitEnqueued) {
+ // We expect one executor failure to trigger many FetchFailures in rapid succession,
+ // but all of those task failures can typically be handled by a single resubmission of
+ // the failed stage. We avoid flooding the scheduler's event queue with resubmit
+ // messages by checking whether a resubmit is already in the event queue for the
+ // failed stage. If there is already a resubmit enqueued for a different failed
+ // stage, that event would also be sufficient to handle the current failed stage, but
+ // producing a resubmit for each failed stage makes debugging and logging a little
+ // simpler while not producing an overwhelming number of scheduler events.
+ logInfo(
+ s"Resubmitting $mapStage (${mapStage.name}) and " +
+ s"$failedStage (${failedStage.name}) due to fetch failure"
+ )
+ messageScheduler.schedule(
+ new Runnable {
+ override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages)
+ },
+ DAGScheduler.RESUBMIT_TIMEOUT,
+ TimeUnit.MILLISECONDS
+ )
+ }
}
// Mark the map whose fetch failed as broken in the map stage
if (mapId != -1) {
@@ -1661,7 +1681,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler
} catch {
case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t)
}
- dagScheduler.sc.stop()
+ dagScheduler.sc.stopInNewThread()
}
override def onStop(): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala
index 20ab27d127..70553d8be2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala
@@ -25,26 +25,30 @@ import scala.collection.mutable.HashMap
private[scheduler] class ExecutorFailuresInTaskSet(val node: String) {
/**
* Mapping from index of the tasks in the taskset, to the number of times it has failed on this
- * executor.
+ * executor and the most recent failure time.
*/
- val taskToFailureCount = HashMap[Int, Int]()
+ val taskToFailureCountAndFailureTime = HashMap[Int, (Int, Long)]()
- def updateWithFailure(taskIndex: Int): Unit = {
- val prevFailureCount = taskToFailureCount.getOrElse(taskIndex, 0)
- taskToFailureCount(taskIndex) = prevFailureCount + 1
+ def updateWithFailure(taskIndex: Int, failureTime: Long): Unit = {
+ val (prevFailureCount, prevFailureTime) =
+ taskToFailureCountAndFailureTime.getOrElse(taskIndex, (0, -1L))
+ // these times always come from the driver, so we don't need to worry about skew, but might
+ // as well still be defensive in case there is non-monotonicity in the clock
+ val newFailureTime = math.max(prevFailureTime, failureTime)
+ taskToFailureCountAndFailureTime(taskIndex) = (prevFailureCount + 1, newFailureTime)
}
- def numUniqueTasksWithFailures: Int = taskToFailureCount.size
+ def numUniqueTasksWithFailures: Int = taskToFailureCountAndFailureTime.size
/**
* Return the number of times this executor has failed on the given task index.
*/
def getNumTaskFailures(index: Int): Int = {
- taskToFailureCount.getOrElse(index, 0)
+ taskToFailureCountAndFailureTime.getOrElse(index, (0, 0))._1
}
override def toString(): String = {
s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " +
- s"tasksToFailureCount = $taskToFailureCount"
+ s"tasksToFailureCount = $taskToFailureCountAndFailureTime"
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index d19353f2a9..6abdf0fd53 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -42,7 +42,8 @@ import org.apache.spark.rdd.RDD
* @param outputId index of the task in this job (a job can launch tasks on only a subset of the
* input RDD's partitions).
* @param localProperties copy of thread-local properties set by the user on the driver side.
- * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
+ * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side
+ * and sent to executor side.
*
* The parameters below are optional:
* @param jobId id of the job this task belongs to
@@ -57,12 +58,12 @@ private[spark] class ResultTask[T, U](
locs: Seq[TaskLocation],
val outputId: Int,
localProperties: Properties,
- metrics: TaskMetrics,
+ serializedTaskMetrics: Array[Byte],
jobId: Option[Int] = None,
appId: Option[String] = None,
appAttemptId: Option[String] = None)
- extends Task[U](stageId, stageAttemptId, partition.index, metrics, localProperties, jobId,
- appId, appAttemptId)
+ extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics,
+ jobId, appId, appAttemptId)
with Serializable {
@transient private[this] val preferredLocs: Seq[TaskLocation] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 31011de85b..994b81e062 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -42,8 +42,9 @@ import org.apache.spark.shuffle.ShuffleWriter
* the type should be (RDD[_], ShuffleDependency[_, _, _]).
* @param partition partition of the RDD this task is associated with
* @param locs preferred task execution locations for locality scheduling
- * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
* @param localProperties copy of thread-local properties set by the user on the driver side.
+ * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side
+ * and sent to executor side.
*
* The parameters below are optional:
* @param jobId id of the job this task belongs to
@@ -56,18 +57,18 @@ private[spark] class ShuffleMapTask(
taskBinary: Broadcast[Array[Byte]],
partition: Partition,
@transient private var locs: Seq[TaskLocation],
- metrics: TaskMetrics,
localProperties: Properties,
+ serializedTaskMetrics: Array[Byte],
jobId: Option[Int] = None,
appId: Option[String] = None,
appAttemptId: Option[String] = None)
- extends Task[MapStatus](stageId, stageAttemptId, partition.index, metrics, localProperties, jobId,
- appId, appAttemptId)
+ extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties,
+ serializedTaskMetrics, jobId, appId, appAttemptId)
with Logging {
/** A constructor used only in test suites. This does not require passing in an RDD. */
def this(partitionId: Int) {
- this(0, 0, null, new Partition { override def index: Int = 0 }, null, null, new Properties)
+ this(0, 0, null, new Partition { override def index: Int = 0 }, null, new Properties, null)
}
@transient private val preferredLocs: Seq[TaskLocation] = {
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 1554200aea..5becca6c06 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -48,6 +48,8 @@ import org.apache.spark.util._
* @param partitionId index of the number in the RDD
* @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
* @param localProperties copy of thread-local properties set by the user on the driver side.
+ * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side
+ * and sent to executor side.
*
* The parameters below are optional:
* @param jobId id of the job this task belongs to
@@ -58,13 +60,17 @@ private[spark] abstract class Task[T](
val stageId: Int,
val stageAttemptId: Int,
val partitionId: Int,
- // The default value is only used in tests.
- val metrics: TaskMetrics = TaskMetrics.registered,
@transient var localProperties: Properties = new Properties,
+ // The default value is only used in tests.
+ serializedTaskMetrics: Array[Byte] =
+ SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array(),
val jobId: Option[Int] = None,
val appId: Option[String] = None,
val appAttemptId: Option[String] = None) extends Serializable {
+ @transient lazy val metrics: TaskMetrics =
+ SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics))
+
/**
* Called by [[org.apache.spark.executor.Executor]] to run this task.
*
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 b03cfe4f0d..9a8e313f9e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -51,13 +51,28 @@ import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils}
* acquire a lock on us, so we need to make sure that we don't try to lock the backend while
* we are holding a lock on ourselves.
*/
-private[spark] class TaskSchedulerImpl(
+private[spark] class TaskSchedulerImpl private[scheduler](
val sc: SparkContext,
val maxTaskFailures: Int,
+ blacklistTrackerOpt: Option[BlacklistTracker],
isLocal: Boolean = false)
extends TaskScheduler with Logging
{
- def this(sc: SparkContext) = this(sc, sc.conf.get(config.MAX_TASK_FAILURES))
+
+ def this(sc: SparkContext) = {
+ this(
+ sc,
+ sc.conf.get(config.MAX_TASK_FAILURES),
+ TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf))
+ }
+
+ def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = {
+ this(
+ sc,
+ maxTaskFailures,
+ TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf),
+ isLocal = isLocal)
+ }
val conf = sc.conf
@@ -209,7 +224,7 @@ private[spark] class TaskSchedulerImpl(
private[scheduler] def createTaskSetManager(
taskSet: TaskSet,
maxTaskFailures: Int): TaskSetManager = {
- new TaskSetManager(this, taskSet, maxTaskFailures)
+ new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt)
}
override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized {
@@ -256,6 +271,8 @@ private[spark] class TaskSchedulerImpl(
availableCpus: Array[Int],
tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : Boolean = {
var launchedTask = false
+ // nodes and executors that are blacklisted for the entire application have already been
+ // filtered out by this point
for (i <- 0 until shuffledOffers.size) {
val execId = shuffledOffers(i).executorId
val host = shuffledOffers(i).host
@@ -308,8 +325,20 @@ private[spark] class TaskSchedulerImpl(
}
}
+ // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do
+ // this here to avoid a separate thread and added synchronization overhead, and also because
+ // updating the blacklist is only relevant when task offers are being made.
+ blacklistTrackerOpt.foreach(_.applyBlacklistTimeout())
+
+ val filteredOffers = blacklistTrackerOpt.map { blacklistTracker =>
+ offers.filter { offer =>
+ !blacklistTracker.isNodeBlacklisted(offer.host) &&
+ !blacklistTracker.isExecutorBlacklisted(offer.executorId)
+ }
+ }.getOrElse(offers)
+
// Randomly shuffle offers to avoid always placing tasks on the same set of workers.
- val shuffledOffers = Random.shuffle(offers)
+ val shuffledOffers = Random.shuffle(filteredOffers)
// Build a list of tasks to assign to each worker.
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores))
val availableCpus = shuffledOffers.map(o => o.cores).toArray
@@ -574,6 +603,7 @@ private[spark] class TaskSchedulerImpl(
executorIdToHost -= executorId
rootPool.executorLost(executorId, host, reason)
}
+ blacklistTrackerOpt.foreach(_.handleRemovedExecutor(executorId))
}
def executorAdded(execId: String, host: String) {
@@ -600,6 +630,14 @@ private[spark] class TaskSchedulerImpl(
executorIdToRunningTaskIds.get(execId).exists(_.nonEmpty)
}
+ /**
+ * Get a snapshot of the currently blacklisted nodes for the entire application. This is
+ * thread-safe -- it can be called without a lock on the TaskScheduler.
+ */
+ def nodeBlacklist(): scala.collection.immutable.Set[String] = {
+ blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set())
+ }
+
// By default, rack is unknown
def getRackForHost(value: String): Option[String] = None
@@ -678,4 +716,13 @@ private[spark] object TaskSchedulerImpl {
retval.toList
}
+
+ private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = {
+ if (BlacklistTracker.isBlacklistEnabled(conf)) {
+ Some(new BlacklistTracker(conf))
+ } else {
+ None
+ }
+ }
+
}
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 f4b0f55b76..e815b7e0cf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
@@ -28,6 +28,10 @@ import org.apache.spark.util.Clock
* (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes
* for the entire taskset.
*
+ * It also must store sufficient information in task failures for application level blacklisting,
+ * which is handled by [[BlacklistTracker]]. Note that BlacklistTracker does not know anything
+ * about task failures until a taskset completes successfully.
+ *
* THREADING: This class is a helper to [[TaskSetManager]]; as with the methods in
* [[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.
@@ -41,7 +45,9 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE)
/**
- * A map from each executor to the task failures on that executor.
+ * A map from each executor to the task failures on that executor. This is used for blacklisting
+ * within this taskset, and it is also relayed onto [[BlacklistTracker]] for app-level
+ * blacklisting if this taskset completes successfully.
*/
val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]()
@@ -57,9 +63,9 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
/**
* Return true if this executor is blacklisted for the given task. This does *not*
- * need to return true if the executor is blacklisted for the entire stage.
- * That is to keep this method as fast as possible in the inner-loop of the
- * scheduler, where those filters will have already been applied.
+ * need to return true if the executor is blacklisted for the entire stage, or blacklisted
+ * for the entire application. That is to keep this method as fast as possible in the inner-loop
+ * of the scheduler, where those filters will have already been applied.
*/
def isExecutorBlacklistedForTask(executorId: String, index: Int): Boolean = {
execToFailures.get(executorId).exists { execFailures =>
@@ -72,10 +78,10 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
}
/**
- * Return true if this executor is blacklisted for the given stage. Completely ignores
- * anything to do with the node the executor is on. That
- * is to keep this method as fast as possible in the inner-loop of the scheduler, where those
- * filters will already have been applied.
+ * Return true if this executor is blacklisted for the given stage. Completely ignores whether
+ * the executor is blacklisted for the entire application (or anything to do with the node the
+ * executor is on). That is to keep this method as fast as possible in the inner-loop of the
+ * scheduler, where those filters will already have been applied.
*/
def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = {
blacklistedExecs.contains(executorId)
@@ -90,7 +96,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
exec: String,
index: Int): Unit = {
val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host))
- execFailures.updateWithFailure(index)
+ execFailures.updateWithFailure(index, clock.getTimeMillis())
// check if this task has also failed on other executors on the same host -- if its gone
// over the limit, blacklist this task from the entire host.
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 f2a432cad3..3756c216f5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -51,6 +51,7 @@ private[spark] class TaskSetManager(
sched: TaskSchedulerImpl,
val taskSet: TaskSet,
val maxTaskFailures: Int,
+ blacklistTracker: Option[BlacklistTracker] = None,
clock: Clock = new SystemClock()) extends Schedulable with Logging {
private val conf = sched.sc.conf
@@ -85,10 +86,8 @@ private[spark] class TaskSetManager(
var calculatedTasks = 0
private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = {
- if (BlacklistTracker.isBlacklistEnabled(conf)) {
- Some(new TaskSetBlacklist(conf, stageId, clock))
- } else {
- None
+ blacklistTracker.map { _ =>
+ new TaskSetBlacklist(conf, stageId, clock)
}
}
@@ -487,6 +486,12 @@ private[spark] class TaskSetManager(
private def maybeFinishTaskSet() {
if (isZombie && runningTasks == 0) {
sched.taskSetFinished(this)
+ if (tasksSuccessful == numTasks) {
+ blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(
+ taskSet.stageId,
+ taskSet.stageAttemptId,
+ taskSetBlacklistHelperOpt.get.execToFailures))
+ }
}
}
@@ -589,6 +594,7 @@ private[spark] class TaskSetManager(
private[scheduler] def abortIfCompletelyBlacklisted(
hostToExecutors: HashMap[String, HashSet[String]]): Unit = {
taskSetBlacklistHelperOpt.foreach { taskSetBlacklist =>
+ val appBlacklist = blacklistTracker.get
// Only look for unschedulable tasks when at least one executor has registered. Otherwise,
// task sets will be (unnecessarily) aborted in cases when no executors have registered yet.
if (hostToExecutors.nonEmpty) {
@@ -615,13 +621,15 @@ private[spark] class TaskSetManager(
val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) =>
// Check if the task can run on the node
val nodeBlacklisted =
- taskSetBlacklist.isNodeBlacklistedForTaskSet(host) ||
- taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet)
+ appBlacklist.isNodeBlacklisted(host) ||
+ taskSetBlacklist.isNodeBlacklistedForTaskSet(host) ||
+ taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet)
if (nodeBlacklisted) {
true
} else {
// Check if the task can run on any of the executors
execsOnHost.forall { exec =>
+ appBlacklist.isExecutorBlacklisted(exec) ||
taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) ||
taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
index 0a4f19d760..0280359809 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
@@ -99,7 +99,8 @@ private[spark] object CoarseGrainedClusterMessages {
case class RequestExecutors(
requestedTotal: Int,
localityAwareTasks: Int,
- hostToLocalTaskCount: Map[String, Int])
+ hostToLocalTaskCount: Map[String, Int],
+ nodeBlacklist: Set[String])
extends CoarseGrainedClusterMessage
// Check if an executor was force-killed but for a reason unrelated to the running tasks.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 368cd30a2e..7befdb0c1f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -139,7 +139,7 @@ private[spark] class StandaloneSchedulerBackend(
scheduler.error(reason)
} finally {
// Ensure the application terminates, as we can no longer run jobs.
- sc.stop()
+ sc.stopInNewThread()
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala
index 8e3436f134..cdd3b8d851 100644
--- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala
+++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala
@@ -21,12 +21,15 @@ import java.util.Properties
import javax.crypto.KeyGenerator
import javax.crypto.spec.{IvParameterSpec, SecretKeySpec}
+import scala.collection.JavaConverters._
+
import org.apache.commons.crypto.random._
import org.apache.commons.crypto.stream._
import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
+import org.apache.spark.network.util.CryptoUtils
/**
* A util class for manipulating IO encryption and decryption streams.
@@ -37,8 +40,6 @@ private[spark] object CryptoStreamUtils extends Logging {
val IV_LENGTH_IN_BYTES = 16
// The prefix of IO encryption related configurations in Spark configuration.
val SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX = "spark.io.encryption.commons.config."
- // The prefix for the configurations passing to Apache Commons Crypto library.
- val COMMONS_CRYPTO_CONF_PREFIX = "commons.crypto."
/**
* Helper method to wrap `OutputStream` with `CryptoOutputStream` for encryption.
@@ -70,18 +71,9 @@ private[spark] object CryptoStreamUtils extends Logging {
new SecretKeySpec(key, "AES"), new IvParameterSpec(iv))
}
- /**
- * Get Commons-crypto configurations from Spark configurations identified by prefix.
- */
def toCryptoConf(conf: SparkConf): Properties = {
- val props = new Properties()
- conf.getAll.foreach { case (k, v) =>
- if (k.startsWith(SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX)) {
- props.put(COMMONS_CRYPTO_CONF_PREFIX + k.substring(
- SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX.length()), v)
- }
- }
- props
+ CryptoUtils.toCryptoConf(SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX,
+ conf.getAll.toMap.asJava.entrySet())
}
/**
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 b9d83495d2..8b2e26cdd9 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -42,24 +42,21 @@ private[spark] class BlockStoreShuffleReader[K, C](
/** Read the combined key-values for this reduce task */
override def read(): Iterator[Product2[K, C]] = {
- val blockFetcherItr = new ShuffleBlockFetcherIterator(
+ val wrappedStreams = new ShuffleBlockFetcherIterator(
context,
blockManager.shuffleClient,
blockManager,
mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition),
+ serializerManager.wrapStream,
// Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
- SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue))
-
- // Wrap the streams for compression and encryption based on configuration
- val wrappedStreams = blockFetcherItr.map { case (blockId, inputStream) =>
- serializerManager.wrapStream(blockId, inputStream)
- }
+ SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
+ SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
val serializerInstance = dep.serializer.newInstance()
// Create a key/value iterator for each stream
- val recordIter = wrappedStreams.flatMap { wrappedStream =>
+ val recordIter = wrappedStreams.flatMap { case (blockId, wrappedStream) =>
// Note: the asKeyValueIterator below wraps a key/value iterator inside of a
// NextIterator. The NextIterator makes sure that close() is called on the
// underlying InputStream when all records have been read.
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 269c12d6da..b720aaee7c 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -17,19 +17,21 @@
package org.apache.spark.storage
-import java.io.InputStream
+import java.io.{InputStream, IOException}
+import java.nio.ByteBuffer
import java.util.concurrent.LinkedBlockingQueue
import javax.annotation.concurrent.GuardedBy
+import scala.collection.mutable
import scala.collection.mutable.{ArrayBuffer, HashSet, Queue}
-import scala.util.control.NonFatal
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.internal.Logging
-import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util.Utils
+import org.apache.spark.util.io.ChunkedByteBufferOutputStream
/**
* An iterator that fetches multiple blocks. For local blocks, it fetches from the local block
@@ -47,8 +49,10 @@ import org.apache.spark.util.Utils
* @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.
+ * @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.
+ * @param detectCorrupt whether to detect any corruption in fetched blocks.
*/
private[spark]
final class ShuffleBlockFetcherIterator(
@@ -56,8 +60,10 @@ final class ShuffleBlockFetcherIterator(
shuffleClient: ShuffleClient,
blockManager: BlockManager,
blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
+ streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
- maxReqsInFlight: Int)
+ maxReqsInFlight: Int,
+ detectCorrupt: Boolean)
extends Iterator[(BlockId, InputStream)] with Logging {
import ShuffleBlockFetcherIterator._
@@ -94,7 +100,7 @@ final class ShuffleBlockFetcherIterator(
* Current [[FetchResult]] being processed. We track this so we can release the current buffer
* in case of a runtime exception when processing the current buffer.
*/
- @volatile private[this] var currentResult: FetchResult = null
+ @volatile private[this] var currentResult: SuccessFetchResult = null
/**
* Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that
@@ -108,6 +114,12 @@ final class ShuffleBlockFetcherIterator(
/** Current number of requests in flight */
private[this] var reqsInFlight = 0
+ /**
+ * The blocks that can't be decompressed successfully, it is used to guarantee that we retry
+ * at most once for those corrupted blocks.
+ */
+ private[this] val corruptedBlocks = mutable.HashSet[BlockId]()
+
private[this] val shuffleMetrics = context.taskMetrics().createTempShuffleReadMetrics()
/**
@@ -123,9 +135,8 @@ final class ShuffleBlockFetcherIterator(
// The currentResult is set to null to prevent releasing the buffer again on cleanup()
private[storage] def releaseCurrentResultBuffer(): Unit = {
// Release the current buffer if necessary
- currentResult match {
- case SuccessFetchResult(_, _, _, buf, _) => buf.release()
- case _ =>
+ if (currentResult != null) {
+ currentResult.buf.release()
}
currentResult = null
}
@@ -305,40 +316,84 @@ final class ShuffleBlockFetcherIterator(
*/
override def next(): (BlockId, InputStream) = {
numBlocksProcessed += 1
- val startFetchWait = System.currentTimeMillis()
- currentResult = results.take()
- val result = currentResult
- val stopFetchWait = System.currentTimeMillis()
- shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait)
-
- result match {
- case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) =>
- if (address != blockManager.blockManagerId) {
- shuffleMetrics.incRemoteBytesRead(buf.size)
- shuffleMetrics.incRemoteBlocksFetched(1)
- }
- bytesInFlight -= size
- if (isNetworkReqDone) {
- reqsInFlight -= 1
- logDebug("Number of requests in flight " + reqsInFlight)
- }
- case _ =>
- }
- // Send fetch requests up to maxBytesInFlight
- fetchUpToMaxBytes()
- result match {
- case FailureFetchResult(blockId, address, e) =>
- throwFetchFailedException(blockId, address, e)
+ var result: FetchResult = null
+ var input: InputStream = null
+ // Take the next fetched result and try to decompress it to detect data corruption,
+ // then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch
+ // is also corrupt, so the previous stage could be retried.
+ // For local shuffle block, throw FailureFetchResult for the first IOException.
+ while (result == null) {
+ val startFetchWait = System.currentTimeMillis()
+ result = results.take()
+ val stopFetchWait = System.currentTimeMillis()
+ shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait)
- case SuccessFetchResult(blockId, address, _, buf, _) =>
- try {
- (result.blockId, new BufferReleasingInputStream(buf.createInputStream(), this))
- } catch {
- case NonFatal(t) =>
- throwFetchFailedException(blockId, address, t)
- }
+ result match {
+ case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) =>
+ if (address != blockManager.blockManagerId) {
+ shuffleMetrics.incRemoteBytesRead(buf.size)
+ shuffleMetrics.incRemoteBlocksFetched(1)
+ }
+ bytesInFlight -= size
+ if (isNetworkReqDone) {
+ reqsInFlight -= 1
+ logDebug("Number of requests in flight " + reqsInFlight)
+ }
+
+ val in = try {
+ buf.createInputStream()
+ } catch {
+ // The exception could only be throwed by local shuffle block
+ case e: IOException =>
+ assert(buf.isInstanceOf[FileSegmentManagedBuffer])
+ logError("Failed to create input stream from local block", e)
+ buf.release()
+ throwFetchFailedException(blockId, address, e)
+ }
+
+ input = streamWrapper(blockId, in)
+ // Only copy the stream if it's wrapped by compression or encryption, also the size of
+ // block is small (the decompressed block is smaller than maxBytesInFlight)
+ if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) {
+ val originalInput = input
+ val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate)
+ try {
+ // Decompress the whole block at once to detect any corruption, which could increase
+ // the memory usage tne potential increase the chance of OOM.
+ // TODO: manage the memory used here, and spill it into disk in case of OOM.
+ Utils.copyStream(input, out)
+ out.close()
+ input = out.toChunkedByteBuffer.toInputStream(dispose = true)
+ } catch {
+ case e: IOException =>
+ buf.release()
+ if (buf.isInstanceOf[FileSegmentManagedBuffer]
+ || corruptedBlocks.contains(blockId)) {
+ throwFetchFailedException(blockId, address, e)
+ } else {
+ logWarning(s"got an corrupted block $blockId from $address, fetch again", e)
+ corruptedBlocks += blockId
+ fetchRequests += FetchRequest(address, Array((blockId, size)))
+ result = null
+ }
+ } finally {
+ // TODO: release the buf here to free memory earlier
+ originalInput.close()
+ in.close()
+ }
+ }
+
+ case FailureFetchResult(blockId, address, e) =>
+ throwFetchFailedException(blockId, address, e)
+ }
+
+ // Send fetch requests up to maxBytesInFlight
+ fetchUpToMaxBytes()
}
+
+ currentResult = result.asInstanceOf[SuccessFetchResult]
+ (currentResult.blockId, new BufferReleasingInputStream(input, this))
}
private def fetchUpToMaxBytes(): Unit = {
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 fff21218b1..fb54dd66a3 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
@@ -331,7 +331,15 @@ private[spark] class MemoryStore(
var unrollMemoryUsedByThisBlock = 0L
// Underlying buffer for unrolling the block
val redirectableStream = new RedirectableOutputStream
- val bbos = new ChunkedByteBufferOutputStream(initialMemoryThreshold.toInt, allocator)
+ 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 " +
+ s"${Utils.bytesToString(Int.MaxValue)}")
+ Int.MaxValue
+ } else {
+ initialMemoryThreshold.toInt
+ }
+ val bbos = new ChunkedByteBufferOutputStream(chunkSize, allocator)
redirectableStream.setOutputStream(bbos)
val serializationStream: SerializationStream = {
val autoPick = !blockId.isInstanceOf[StreamBlockId]
@@ -694,7 +702,7 @@ private[storage] class PartiallyUnrolledIterator[T](
}
override def next(): T = {
- if (unrolled == null) {
+ if (unrolled == null || !unrolled.hasNext) {
rest.next()
} else {
unrolled.next()
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index b828532aba..7d31ac54a7 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -60,6 +60,8 @@ private[spark] class SparkUI private (
var appId: String = _
+ private var streamingJobProgressListener: Option[SparkListener] = None
+
/** Initialize all components of the server. */
def initialize() {
val jobsTab = new JobsTab(this)
@@ -124,6 +126,12 @@ private[spark] class SparkUI private (
def getApplicationInfo(appId: String): Option[ApplicationInfo] = {
getApplicationInfoList.find(_.id == appId)
}
+
+ def getStreamingJobProgressListener: Option[SparkListener] = streamingJobProgressListener
+
+ def setStreamingJobProgressListener(sparkListener: SparkListener): Unit = {
+ streamingJobProgressListener = Option(sparkListener)
+ }
}
private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String)
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index 8c80155867..b8604c52e6 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -147,10 +147,7 @@ private[spark] abstract class WebUI(
}
/** Return the url of web interface. Only valid after bind(). */
- def webUrl: String = {
- val protocol = if (sslOptions.enabled) "https" else "http"
- s"$protocol://$publicHostName:$boundPort"
- }
+ def webUrl: String = s"http://$publicHostName:$boundPort"
/** Return the actual port to which this server is bound. Only valid after bind(). */
def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1)
diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
index 60a6e82c6f..1aa4456ed0 100644
--- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
@@ -19,7 +19,7 @@ package org.apache.spark.util
import java.util.concurrent._
-import scala.concurrent.{Await, Awaitable, ExecutionContext, ExecutionContextExecutor}
+import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor}
import scala.concurrent.duration.Duration
import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread}
import scala.util.control.NonFatal
@@ -180,39 +180,30 @@ private[spark] object ThreadUtils {
// scalastyle:off awaitresult
/**
- * Preferred alternative to `Await.result()`. This method wraps and re-throws any exceptions
- * thrown by the underlying `Await` call, ensuring that this thread's stack trace appears in
- * logs.
- */
- @throws(classOf[SparkException])
- def awaitResult[T](awaitable: Awaitable[T], atMost: Duration): T = {
- try {
- Await.result(awaitable, atMost)
- // scalastyle:on awaitresult
- } catch {
- case NonFatal(t) =>
- throw new SparkException("Exception thrown in awaitResult: ", t)
- }
- }
-
- /**
- * Calls `Awaitable.result` directly to avoid using `ForkJoinPool`'s `BlockingContext`, wraps
- * and re-throws any exceptions with nice stack track.
+ * Preferred alternative to `Await.result()`.
+ *
+ * This method wraps and re-throws any exceptions thrown by the underlying `Await` call, ensuring
+ * that this thread's stack trace appears in logs.
*
- * Codes running in the user's thread may be in a thread of Scala ForkJoinPool. As concurrent
- * executions in ForkJoinPool may see some [[ThreadLocal]] value unexpectedly, this method
- * basically prevents ForkJoinPool from running other tasks in the current waiting thread.
+ * In addition, it calls `Awaitable.result` directly to avoid using `ForkJoinPool`'s
+ * `BlockingContext`. Codes running in the user's thread may be in a thread of Scala ForkJoinPool.
+ * As concurrent executions in ForkJoinPool may see some [[ThreadLocal]] value unexpectedly, this
+ * method basically prevents ForkJoinPool from running other tasks in the current waiting thread.
+ * In general, we should use this method because many places in Spark use [[ThreadLocal]] and it's
+ * hard to debug when [[ThreadLocal]]s leak to other tasks.
*/
@throws(classOf[SparkException])
- def awaitResultInForkJoinSafely[T](awaitable: Awaitable[T], atMost: Duration): T = {
+ def awaitResult[T](awaitable: Awaitable[T], atMost: Duration): T = {
try {
// `awaitPermission` is not actually used anywhere so it's safe to pass in null here.
// See SPARK-13747.
val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait]
- awaitable.result(Duration.Inf)(awaitPermission)
+ awaitable.result(atMost)(awaitPermission)
} catch {
- case NonFatal(t) =>
+ // TimeoutException is thrown in the current thread, so not need to warp the exception.
+ case NonFatal(t) if !t.isInstanceOf[TimeoutException] =>
throw new SparkException("Exception thrown in awaitResult: ", t)
}
}
+ // scalastyle:on awaitresult
}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 91f5606127..078cc3d5b4 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -18,7 +18,7 @@
package org.apache.spark.util
import java.io._
-import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo}
+import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.net._
import java.nio.ByteBuffer
import java.nio.channels.Channels
@@ -1249,7 +1249,7 @@ private[spark] object Utils extends Logging {
val currentThreadName = Thread.currentThread().getName
if (sc != null) {
logError(s"uncaught error in thread $currentThreadName, stopping SparkContext", t)
- sc.stop()
+ sc.stopInNewThread()
}
if (!NonFatal(t)) {
logError(s"throw uncaught fatal error in thread $currentThreadName", t)
@@ -2131,28 +2131,46 @@ private[spark] object Utils extends Logging {
// We need to filter out null values here because dumpAllThreads() may return null array
// elements for threads that are dead / don't exist.
val threadInfos = ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null)
- threadInfos.sortBy(_.getThreadId).map { case threadInfo =>
- val monitors = threadInfo.getLockedMonitors.map(m => m.getLockedStackFrame -> m).toMap
- val stackTrace = threadInfo.getStackTrace.map { frame =>
- monitors.get(frame) match {
- case Some(monitor) =>
- monitor.getLockedStackFrame.toString + s" => holding ${monitor.lockString}"
- case None =>
- frame.toString
- }
- }.mkString("\n")
-
- // use a set to dedup re-entrant locks that are held at multiple places
- val heldLocks = (threadInfo.getLockedSynchronizers.map(_.lockString)
- ++ threadInfo.getLockedMonitors.map(_.lockString)
- ).toSet
+ threadInfos.sortBy(_.getThreadId).map(threadInfoToThreadStackTrace)
+ }
- ThreadStackTrace(threadInfo.getThreadId, threadInfo.getThreadName, threadInfo.getThreadState,
- stackTrace, if (threadInfo.getLockOwnerId < 0) None else Some(threadInfo.getLockOwnerId),
- Option(threadInfo.getLockInfo).map(_.lockString).getOrElse(""), heldLocks.toSeq)
+ def getThreadDumpForThread(threadId: Long): Option[ThreadStackTrace] = {
+ if (threadId <= 0) {
+ None
+ } else {
+ // The Int.MaxValue here requests the entire untruncated stack trace of the thread:
+ val threadInfo =
+ Option(ManagementFactory.getThreadMXBean.getThreadInfo(threadId, Int.MaxValue))
+ threadInfo.map(threadInfoToThreadStackTrace)
}
}
+ private def threadInfoToThreadStackTrace(threadInfo: ThreadInfo): ThreadStackTrace = {
+ val monitors = threadInfo.getLockedMonitors.map(m => m.getLockedStackFrame -> m).toMap
+ val stackTrace = threadInfo.getStackTrace.map { frame =>
+ monitors.get(frame) match {
+ case Some(monitor) =>
+ monitor.getLockedStackFrame.toString + s" => holding ${monitor.lockString}"
+ case None =>
+ frame.toString
+ }
+ }.mkString("\n")
+
+ // use a set to dedup re-entrant locks that are held at multiple places
+ val heldLocks =
+ (threadInfo.getLockedSynchronizers ++ threadInfo.getLockedMonitors).map(_.lockString).toSet
+
+ ThreadStackTrace(
+ threadId = threadInfo.getThreadId,
+ threadName = threadInfo.getThreadName,
+ threadState = threadInfo.getThreadState,
+ stackTrace = stackTrace,
+ blockedByThreadId =
+ if (threadInfo.getLockOwnerId < 0) None else Some(threadInfo.getLockOwnerId),
+ blockedByLock = Option(threadInfo.getLockInfo).map(_.lockString).getOrElse(""),
+ holdingLocks = heldLocks.toSeq)
+ }
+
/**
* Convert all spark properties set in the given SparkConf to a sequence of java options.
*/
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index da08661d13..7572cac393 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -151,7 +151,7 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
* @param dispose if true, `ChunkedByteBuffer.dispose()` will be called at the end of the stream
* in order to close any memory-mapped files which back the buffer.
*/
-private class ChunkedByteBufferInputStream(
+private[spark] class ChunkedByteBufferInputStream(
var chunkedByteBuffer: ChunkedByteBuffer,
dispose: Boolean)
extends InputStream {
diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
index 297524c943..a7e0075deb 100644
--- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala
@@ -56,11 +56,14 @@ private[spark] object SamplingUtils {
val rand = new XORShiftRandom(seed)
while (input.hasNext) {
val item = input.next()
+ l += 1
+ // There are k elements in the reservoir, and the l-th element has been
+ // consumed. It should be chosen with probability k/l. The expression
+ // below is a random long chosen uniformly from [0,l)
val replacementIndex = (rand.nextDouble() * l).toLong
if (replacementIndex < k) {
reservoir(replacementIndex.toInt) = item
}
- l += 1
}
(reservoir, l)
}
diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
index 682d98867b..0c77123740 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -27,8 +27,10 @@
import org.slf4j.LoggerFactory;
import org.slf4j.bridge.SLF4JBridgeHandler;
import static org.junit.Assert.*;
+import static org.junit.Assume.*;
import org.apache.spark.internal.config.package$;
+import org.apache.spark.util.Utils;
/**
* These tests require the Spark assembly to be built before they can be run.
@@ -155,6 +157,10 @@ public void testRedirectToLog() throws Exception {
@Test
public void testChildProcLauncher() throws Exception {
+ // This test is failed on Windows due to the failure of initiating executors
+ // by the path length limitation. See SPARK-18718.
+ assumeTrue(!Utils.isWindows());
+
SparkSubmitOptionParser opts = new SparkSubmitOptionParser();
Map env = new HashMap<>();
env.put("SPARK_PRINT_LAUNCH_COMMAND", "1");
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 89f0b1cb5b..6538507d40 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -22,6 +22,7 @@ import java.util.zip.GZIPOutputStream
import scala.io.Source
+import org.apache.hadoop.fs.Path
import org.apache.hadoop.io._
import org.apache.hadoop.io.compress.DefaultCodec
import org.apache.hadoop.mapred.{FileAlreadyExistsException, FileSplit, JobConf, TextInputFormat, TextOutputFormat}
@@ -255,7 +256,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
val (infile: String, indata: PortableDataStream) = inRdd.collect.head
// Make sure the name and array match
- assert(infile.contains(outFileName)) // a prefix may get added
+ assert(infile.contains(outFile.toURI.getPath)) // a prefix may get added
assert(indata.toArray === testOutput)
}
@@ -532,7 +533,9 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
.mapPartitionsWithInputSplit { (split, part) =>
Iterator(split.asInstanceOf[FileSplit].getPath.toUri.getPath)
}.collect()
- assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001"))
+ val outPathOne = new Path(outDir, "part-00000").toUri.getPath
+ val outPathTwo = new Path(outDir, "part-00001").toUri.getPath
+ assert(inputPaths.toSet === Set(outPathOne, outPathTwo))
}
test("Get input files via new Hadoop API") {
@@ -546,7 +549,9 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
.mapPartitionsWithInputSplit { (split, part) =>
Iterator(split.asInstanceOf[NewFileSplit].getPath.toUri.getPath)
}.collect()
- assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001"))
+ val outPathOne = new Path(outDir, "part-00000").toUri.getPath
+ val outPathTwo = new Path(outDir, "part-00001").toUri.getPath
+ assert(inputPaths.toSet === Set(outPathOne, outPathTwo))
}
test("spark.files.ignoreCorruptFiles should work both HadoopRDD and NewHadoopRDD") {
diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
index 915d7a1b8b..7b6a2313f9 100644
--- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
@@ -272,7 +272,7 @@ private class FakeSchedulerBackend(
protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = {
clusterManagerEndpoint.ask[Boolean](
- RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount))
+ RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty[String]))
}
protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = {
@@ -291,7 +291,7 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin
def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case RequestExecutors(requestedTotal, _, _) =>
+ case RequestExecutors(requestedTotal, _, _, _) =>
targetNumExecutors = requestedTotal
context.reply(true)
case KillExecutors(executorIds) =>
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index a3490fc79e..99150a1430 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -209,6 +209,83 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
assert(jobB.get() === 100)
}
+ test("task reaper kills JVM if killed tasks keep running for too long") {
+ val conf = new SparkConf()
+ .set("spark.task.reaper.enabled", "true")
+ .set("spark.task.reaper.killTimeout", "5s")
+ sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+
+ // Add a listener to release the semaphore once any tasks are launched.
+ val sem = new Semaphore(0)
+ sc.addSparkListener(new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart) {
+ sem.release()
+ }
+ })
+
+ // jobA is the one to be cancelled.
+ val jobA = Future {
+ sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true)
+ sc.parallelize(1 to 10000, 2).map { i =>
+ while (true) { }
+ }.count()
+ }
+
+ // Block until both tasks of job A have started and cancel job A.
+ sem.acquire(2)
+ // Small delay to ensure tasks actually start executing the task body
+ Thread.sleep(1000)
+
+ sc.clearJobGroup()
+ val jobB = sc.parallelize(1 to 100, 2).countAsync()
+ sc.cancelJobGroup("jobA")
+ val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 15.seconds) }.getCause
+ assert(e.getMessage contains "cancel")
+
+ // Once A is cancelled, job B should finish fairly quickly.
+ assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100)
+ }
+
+ test("task reaper will not kill JVM if spark.task.killTimeout == -1") {
+ val conf = new SparkConf()
+ .set("spark.task.reaper.enabled", "true")
+ .set("spark.task.reaper.killTimeout", "-1")
+ .set("spark.task.reaper.PollingInterval", "1s")
+ .set("spark.deploy.maxExecutorRetries", "1")
+ sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+
+ // Add a listener to release the semaphore once any tasks are launched.
+ val sem = new Semaphore(0)
+ sc.addSparkListener(new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart) {
+ sem.release()
+ }
+ })
+
+ // jobA is the one to be cancelled.
+ val jobA = Future {
+ sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true)
+ sc.parallelize(1 to 2, 2).map { i =>
+ val startTime = System.currentTimeMillis()
+ while (System.currentTimeMillis() < startTime + 10000) { }
+ }.count()
+ }
+
+ // Block until both tasks of job A have started and cancel job A.
+ sem.acquire(2)
+ // Small delay to ensure tasks actually start executing the task body
+ Thread.sleep(1000)
+
+ sc.clearJobGroup()
+ val jobB = sc.parallelize(1 to 100, 2).countAsync()
+ sc.cancelJobGroup("jobA")
+ val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 15.seconds) }.getCause
+ assert(e.getMessage contains "cancel")
+
+ // Once A is cancelled, job B should finish fairly quickly.
+ assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100)
+ }
+
test("two jobs sharing the same stage") {
// sem1: make sure cancel is issued after some tasks are launched
// twoJobsSharingStageSemaphore:
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index a854f5bb9b..e626ed3621 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -29,7 +29,7 @@ import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListene
import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.shuffle.ShuffleWriter
import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId}
-import org.apache.spark.util.MutablePair
+import org.apache.spark.util.{MutablePair, Utils}
abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext {
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index c451c596b0..8fba82de54 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.TextInputFormat
import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat}
import org.scalatest.Matchers._
+import org.apache.spark.scheduler.SparkListener
import org.apache.spark.util.Utils
class SparkContextSuite extends SparkFunSuite with LocalSparkContext {
@@ -451,4 +452,19 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext {
sc.stop()
}
}
+
+ test("register and deregister Spark listener from SparkContext") {
+ sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+ try {
+ val sparkListener1 = new SparkListener { }
+ val sparkListener2 = new SparkListener { }
+ sc.addSparkListener(sparkListener1)
+ sc.addSparkListener(sparkListener2)
+ assert(sc.listenerBus.listeners.contains(sparkListener1))
+ assert(sc.listenerBus.listeners.contains(sparkListener2))
+ sc.removeSparkListener(sparkListener1)
+ assert(!sc.listenerBus.listeners.contains(sparkListener1))
+ assert(sc.listenerBus.listeners.contains(sparkListener2))
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala b/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala
new file mode 100644
index 0000000000..6a979aefe6
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api.r
+
+import org.apache.spark.SparkFunSuite
+
+class JVMObjectTrackerSuite extends SparkFunSuite {
+ test("JVMObjectId does not take null IDs") {
+ intercept[IllegalArgumentException] {
+ JVMObjectId(null)
+ }
+ }
+
+ test("JVMObjectTracker") {
+ val tracker = new JVMObjectTracker
+ assert(tracker.size === 0)
+ withClue("an empty tracker can be cleared") {
+ tracker.clear()
+ }
+ val none = JVMObjectId("none")
+ assert(tracker.get(none) === None)
+ intercept[NoSuchElementException] {
+ tracker(JVMObjectId("none"))
+ }
+
+ val obj1 = new Object
+ val id1 = tracker.addAndGetId(obj1)
+ assert(id1 != null)
+ assert(tracker.size === 1)
+ assert(tracker.get(id1).get.eq(obj1))
+ assert(tracker(id1).eq(obj1))
+
+ val obj2 = new Object
+ val id2 = tracker.addAndGetId(obj2)
+ assert(id1 !== id2)
+ assert(tracker.size === 2)
+ assert(tracker(id2).eq(obj2))
+
+ val Some(obj1Removed) = tracker.remove(id1)
+ assert(obj1Removed.eq(obj1))
+ assert(tracker.get(id1) === None)
+ assert(tracker.size === 1)
+ assert(tracker(id2).eq(obj2))
+
+ val obj3 = new Object
+ val id3 = tracker.addAndGetId(obj3)
+ assert(tracker.size === 2)
+ assert(id3 != id1)
+ assert(id3 != id2)
+ assert(tracker(id3).eq(obj3))
+
+ tracker.clear()
+ assert(tracker.size === 0)
+ assert(tracker.get(id1) === None)
+ assert(tracker.get(id2) === None)
+ assert(tracker.get(id3) === None)
+ }
+}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java b/core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala
similarity index 67%
rename from common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java
rename to core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala
index f15ec8d294..085cc267ca 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java
+++ b/core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala
@@ -15,18 +15,17 @@
* limitations under the License.
*/
-package org.apache.spark.network.util;
+package org.apache.spark.api.r
-import java.util.NoSuchElementException;
+import org.apache.spark.SparkFunSuite
-/** Uses System properties to obtain config values. */
-public class SystemPropertyConfigProvider extends ConfigProvider {
- @Override
- public String get(String name) {
- String value = System.getProperty(name);
- if (value == null) {
- throw new NoSuchElementException(name);
- }
- return value;
+class RBackendSuite extends SparkFunSuite {
+ test("close() clears jvmObjectTracker") {
+ val backend = new RBackend
+ val tracker = backend.jvmObjectTracker
+ val id = tracker.addAndGetId(new Object)
+ backend.close()
+ assert(tracker.get(id) === None)
+ assert(tracker.size === 0)
}
}
diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala
index 973676398a..6646068d50 100644
--- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala
+++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala
@@ -137,6 +137,18 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext {
sc.stop()
}
+ test("Cache broadcast to disk") {
+ val conf = new SparkConf()
+ .setMaster("local")
+ .setAppName("test")
+ .set("spark.memory.useLegacyMode", "true")
+ .set("spark.storage.memoryFraction", "0.0")
+ sc = new SparkContext(conf)
+ val list = List[Int](1, 2, 3, 4)
+ val broadcast = sc.broadcast(list)
+ assert(broadcast.value.sum === 10)
+ }
+
/**
* Verify the persistence of state associated with a TorrentBroadcast in a local-cluster.
*
diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
index c9b3d657c2..f50cb38311 100644
--- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
@@ -142,7 +142,7 @@ private[deploy] object IvyTestUtils {
|}
""".stripMargin
val sourceFile =
- new JavaSourceFromString(new File(dir, className).getAbsolutePath, contents)
+ new JavaSourceFromString(new File(dir, className).toURI.getPath, contents)
createCompiledClass(className, dir, sourceFile, Seq.empty)
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 6268880229..9417930d02 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -461,7 +461,7 @@ class SparkSubmitSuite
val tempDir = Utils.createTempDir()
val srcDir = new File(tempDir, "sparkrtest")
srcDir.mkdirs()
- val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").getAbsolutePath,
+ val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath,
"""package sparkrtest;
|
|public class DummyClass implements java.io.Serializable {
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 2c41c432d1..027f412c75 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -66,7 +66,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
}
test("Parse application logs") {
- val provider = new FsHistoryProvider(createTestConf())
+ val clock = new ManualClock(12345678)
+ val provider = new FsHistoryProvider(createTestConf(), clock)
// Write a new-style application log.
val newAppComplete = newLogFile("new1", None, inProgress = false)
@@ -109,12 +110,15 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed)))
}
+ // For completed files, lastUpdated would be lastModified time.
list(0) should be (makeAppInfo("new-app-complete", newAppComplete.getName(), 1L, 5L,
newAppComplete.lastModified(), "test", true))
list(1) should be (makeAppInfo("new-complete-lzf", newAppCompressedComplete.getName(),
1L, 4L, newAppCompressedComplete.lastModified(), "test", true))
+
+ // For Inprogress files, lastUpdated would be current loading time.
list(2) should be (makeAppInfo("new-incomplete", newAppIncomplete.getName(), 1L, -1L,
- newAppIncomplete.lastModified(), "test", false))
+ clock.getTimeMillis(), "test", false))
// Make sure the UI can be rendered.
list.foreach { case info =>
@@ -299,6 +303,48 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
assert(!log2.exists())
}
+ test("log cleaner for inProgress files") {
+ val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10)
+ val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20)
+ val maxAge = TimeUnit.SECONDS.toMillis(40)
+ val clock = new ManualClock(0)
+ val provider = new FsHistoryProvider(
+ createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock)
+
+ val log1 = newLogFile("inProgressApp1", None, inProgress = true)
+ writeFile(log1, true, None,
+ SparkListenerApplicationStart(
+ "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1"))
+ )
+
+ clock.setTime(firstFileModifiedTime)
+ provider.checkForLogs()
+
+ val log2 = newLogFile("inProgressApp2", None, inProgress = true)
+ writeFile(log2, true, None,
+ SparkListenerApplicationStart(
+ "inProgressApp2", Some("inProgressApp2"), 23L, "test2", Some("attempt2"))
+ )
+
+ clock.setTime(secondFileModifiedTime)
+ provider.checkForLogs()
+
+ // This should not trigger any cleanup
+ updateAndCheck(provider)(list => list.size should be(2))
+
+ // Should trigger cleanup for first file but not second one
+ clock.setTime(firstFileModifiedTime + maxAge + 1)
+ updateAndCheck(provider)(list => list.size should be(1))
+ assert(!log1.exists())
+ assert(log2.exists())
+
+ // Should cleanup the second file as well.
+ clock.setTime(secondFileModifiedTime + maxAge + 1)
+ updateAndCheck(provider)(list => list.size should be(0))
+ assert(!log1.exists())
+ assert(!log2.exists())
+ }
+
test("Event log copy") {
val provider = new FsHistoryProvider(createTestConf())
val logs = (1 to 2).map { i =>
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 715811a46f..d3b79dd3e3 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -75,7 +75,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
.set("spark.testing", "true")
provider = new FsHistoryProvider(conf)
provider.checkForLogs()
- val securityManager = new SecurityManager(conf)
+ val securityManager = HistoryServer.createSecurityManager(conf)
server = new HistoryServer(conf, provider, securityManager, 18080)
server.initialize()
@@ -288,7 +288,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
provider = new FsHistoryProvider(conf)
provider.checkForLogs()
- val securityManager = new SecurityManager(conf)
+ val securityManager = HistoryServer.createSecurityManager(conf)
server = new HistoryServer(conf, provider, securityManager, 18080)
server.initialize()
@@ -349,6 +349,17 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
}
+ /**
+ * Verify that the security manager needed for the history server can be instantiated
+ * when `spark.authenticate` is `true`, rather than raise an `IllegalArgumentException`.
+ */
+ test("security manager starts with spark.authenticate set") {
+ val conf = new SparkConf()
+ .set("spark.testing", "true")
+ .set(SecurityManager.SPARK_AUTH_CONF, "true")
+ HistoryServer.createSecurityManager(conf)
+ }
+
test("incomplete apps get refreshed") {
implicit val webDriver: WebDriver = new HtmlUnitDriver
@@ -368,7 +379,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
.set("spark.history.cache.window", "250ms")
.remove("spark.testing")
val provider = new FsHistoryProvider(myConf)
- val securityManager = new SecurityManager(myConf)
+ val securityManager = HistoryServer.createSecurityManager(myConf)
sc = new SparkContext("local", "test", myConf)
val logDirUri = logDir.toURI
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
index 683eeeeb6d..742500d87d 100644
--- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
@@ -51,9 +51,11 @@ class ExecutorSuite extends SparkFunSuite {
when(mockEnv.metricsSystem).thenReturn(mockMetricsSystem)
when(mockEnv.memoryManager).thenReturn(mockMemoryManager)
when(mockEnv.closureSerializer).thenReturn(serializer)
+ val fakeTaskMetrics = serializer.newInstance().serialize(TaskMetrics.registered).array()
+
val serializedTask =
Task.serializeWithDependencies(
- new FakeTask(0, 0),
+ new FakeTask(0, 0, Nil, fakeTaskMetrics),
HashMap[String, Long](),
HashMap[String, Long](),
serializer.newInstance())
diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala
index cac15a1dc4..c88cc13654 100644
--- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala
@@ -26,6 +26,7 @@ import org.scalatest.Matchers
import org.scalatest.concurrent.Eventually._
import org.apache.spark._
+import org.apache.spark.util.Utils
class LauncherBackendSuite extends SparkFunSuite with Matchers {
@@ -35,6 +36,8 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers {
tests.foreach { case (name, master) =>
test(s"$name: launcher handle") {
+ // The tests here are failed due to the cmd length limitation up to 8K on Windows.
+ assume(!Utils.isWindows)
testWithMaster(master)
}
}
diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
index f8054f5fd7..a73b300ec2 100644
--- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
@@ -61,7 +61,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext
pw.close()
// Path to tmpFile
- tmpFilePath = "file://" + tmpFile.getAbsolutePath
+ tmpFilePath = tmpFile.toURI.toString
}
after {
@@ -181,7 +181,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext
sc.textFile(tmpFilePath, 4)
.map(key => (key, 1))
.reduceByKey(_ + _)
- .saveAsTextFile("file://" + tmpFile.getAbsolutePath)
+ .saveAsTextFile(tmpFile.toURI.toString)
sc.listenerBus.waitUntilEmpty(500)
assert(inputRead == numRecords)
@@ -197,7 +197,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext
val numPartitions = 2
val cartVector = 0 to 9
val cartFile = new File(tmpDir, getClass.getSimpleName + "_cart.txt")
- val cartFilePath = "file://" + cartFile.getAbsolutePath
+ val cartFilePath = cartFile.toURI.toString
// write files to disk so we can read them later.
sc.parallelize(cartVector).saveAsTextFile(cartFilePath)
diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
index 58664e77d2..b29a53cffe 100644
--- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
@@ -199,10 +199,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim
val f = sc.parallelize(1 to 100, 4)
.mapPartitions(itr => { Thread.sleep(20); itr })
.countAsync()
- val e = intercept[SparkException] {
+ intercept[TimeoutException] {
ThreadUtils.awaitResult(f, Duration(20, "milliseconds"))
}
- assert(e.getCause.isInstanceOf[TimeoutException])
}
private def testAsyncAction[R](action: RDD[Int] => FutureAction[R]): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
index 7293aa9a25..287ae6ff6e 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
@@ -32,109 +32,104 @@ import org.apache.spark._
import org.apache.spark.util.Utils
class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
+ val envCommand = if (Utils.isWindows) {
+ "cmd.exe /C set"
+ } else {
+ "printenv"
+ }
test("basic pipe") {
- if (testCommandAvailable("cat")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ assume(testCommandAvailable("cat"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val piped = nums.pipe(Seq("cat"))
+ val piped = nums.pipe(Seq("cat"))
- val c = piped.collect()
- assert(c.size === 4)
- assert(c(0) === "1")
- assert(c(1) === "2")
- assert(c(2) === "3")
- assert(c(3) === "4")
- } else {
- assert(true)
- }
+ val c = piped.collect()
+ assert(c.size === 4)
+ assert(c(0) === "1")
+ assert(c(1) === "2")
+ assert(c(2) === "3")
+ assert(c(3) === "4")
}
test("basic pipe with tokenization") {
- if (testCommandAvailable("wc")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
-
- // verify that both RDD.pipe(command: String) and RDD.pipe(command: String, env) work good
- for (piped <- Seq(nums.pipe("wc -l"), nums.pipe("wc -l", Map[String, String]()))) {
- val c = piped.collect()
- assert(c.size === 2)
- assert(c(0).trim === "2")
- assert(c(1).trim === "2")
- }
- } else {
- assert(true)
+ assume(testCommandAvailable("wc"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+
+ // verify that both RDD.pipe(command: String) and RDD.pipe(command: String, env) work good
+ for (piped <- Seq(nums.pipe("wc -l"), nums.pipe("wc -l", Map[String, String]()))) {
+ val c = piped.collect()
+ assert(c.size === 2)
+ assert(c(0).trim === "2")
+ assert(c(1).trim === "2")
}
}
test("failure in iterating over pipe input") {
- if (testCommandAvailable("cat")) {
- val nums =
- sc.makeRDD(Array(1, 2, 3, 4), 2)
- .mapPartitionsWithIndex((index, iterator) => {
- new Iterator[Int] {
- def hasNext = true
- def next() = {
- throw new SparkException("Exception to simulate bad scenario")
- }
- }
- })
-
- val piped = nums.pipe(Seq("cat"))
-
- intercept[SparkException] {
- piped.collect()
- }
+ assume(testCommandAvailable("cat"))
+ val nums =
+ sc.makeRDD(Array(1, 2, 3, 4), 2)
+ .mapPartitionsWithIndex((index, iterator) => {
+ new Iterator[Int] {
+ def hasNext = true
+ def next() = {
+ throw new SparkException("Exception to simulate bad scenario")
+ }
+ }
+ })
+
+ val piped = nums.pipe(Seq("cat"))
+
+ intercept[SparkException] {
+ piped.collect()
}
}
test("advanced pipe") {
- if (testCommandAvailable("cat")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val bl = sc.broadcast(List("0"))
-
- val piped = nums.pipe(Seq("cat"),
+ assume(testCommandAvailable("cat"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val bl = sc.broadcast(List("0"))
+
+ val piped = nums.pipe(Seq("cat"),
+ Map[String, String](),
+ (f: String => Unit) => {
+ bl.value.foreach(f); f("\u0001")
+ },
+ (i: Int, f: String => Unit) => f(i + "_"))
+
+ val c = piped.collect()
+
+ assert(c.size === 8)
+ assert(c(0) === "0")
+ assert(c(1) === "\u0001")
+ assert(c(2) === "1_")
+ assert(c(3) === "2_")
+ assert(c(4) === "0")
+ assert(c(5) === "\u0001")
+ assert(c(6) === "3_")
+ assert(c(7) === "4_")
+
+ val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2)
+ val d = nums1.groupBy(str => str.split("\t")(0)).
+ pipe(Seq("cat"),
Map[String, String](),
(f: String => Unit) => {
bl.value.foreach(f); f("\u0001")
},
- (i: Int, f: String => Unit) => f(i + "_"))
-
- val c = piped.collect()
-
- assert(c.size === 8)
- assert(c(0) === "0")
- assert(c(1) === "\u0001")
- assert(c(2) === "1_")
- assert(c(3) === "2_")
- assert(c(4) === "0")
- assert(c(5) === "\u0001")
- assert(c(6) === "3_")
- assert(c(7) === "4_")
-
- val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2)
- val d = nums1.groupBy(str => str.split("\t")(0)).
- pipe(Seq("cat"),
- Map[String, String](),
- (f: String => Unit) => {
- bl.value.foreach(f); f("\u0001")
- },
- (i: Tuple2[String, Iterable[String]], f: String => Unit) => {
- for (e <- i._2) {
- f(e + "_")
- }
- }).collect()
- assert(d.size === 8)
- assert(d(0) === "0")
- assert(d(1) === "\u0001")
- assert(d(2) === "b\t2_")
- assert(d(3) === "b\t4_")
- assert(d(4) === "0")
- assert(d(5) === "\u0001")
- assert(d(6) === "a\t1_")
- assert(d(7) === "a\t3_")
- } else {
- assert(true)
- }
+ (i: Tuple2[String, Iterable[String]], f: String => Unit) => {
+ for (e <- i._2) {
+ f(e + "_")
+ }
+ }).collect()
+ assert(d.size === 8)
+ assert(d(0) === "0")
+ assert(d(1) === "\u0001")
+ assert(d(2) === "b\t2_")
+ assert(d(3) === "b\t4_")
+ assert(d(4) === "0")
+ assert(d(5) === "\u0001")
+ assert(d(6) === "a\t1_")
+ assert(d(7) === "a\t3_")
}
test("pipe with empty partition") {
@@ -142,67 +137,67 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
val piped = data.pipe("wc -c")
assert(piped.count == 8)
val charCounts = piped.map(_.trim.toInt).collect().toSet
- assert(Set(0, 4, 5) == charCounts)
+ val expected = if (Utils.isWindows) {
+ // Note that newline character on Windows is \r\n which are two.
+ Set(0, 5, 6)
+ } else {
+ Set(0, 4, 5)
+ }
+ assert(expected == charCounts)
}
test("pipe with env variable") {
- if (testCommandAvailable("printenv")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA"))
- val c = piped.collect()
- assert(c.size === 2)
- assert(c(0) === "LALALA")
- assert(c(1) === "LALALA")
- } else {
- assert(true)
- }
+ assume(testCommandAvailable(envCommand))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val piped = nums.pipe(s"$envCommand MY_TEST_ENV", Map("MY_TEST_ENV" -> "LALALA"))
+ val c = piped.collect()
+ assert(c.length === 2)
+ // On Windows, `cmd.exe /C set` is used which prints out it as `varname=value` format
+ // whereas `printenv` usually prints out `value`. So, `varname=` is stripped here for both.
+ assert(c(0).stripPrefix("MY_TEST_ENV=") === "LALALA")
+ assert(c(1).stripPrefix("MY_TEST_ENV=") === "LALALA")
}
test("pipe with process which cannot be launched due to bad command") {
- if (!testCommandAvailable("some_nonexistent_command")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val command = Seq("some_nonexistent_command")
- val piped = nums.pipe(command)
- val exception = intercept[SparkException] {
- piped.collect()
- }
- assert(exception.getMessage.contains(command.mkString(" ")))
+ assume(!testCommandAvailable("some_nonexistent_command"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val command = Seq("some_nonexistent_command")
+ val piped = nums.pipe(command)
+ val exception = intercept[SparkException] {
+ piped.collect()
}
+ assert(exception.getMessage.contains(command.mkString(" ")))
}
test("pipe with process which is launched but fails with non-zero exit status") {
- if (testCommandAvailable("cat")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val command = Seq("cat", "nonexistent_file")
- val piped = nums.pipe(command)
- val exception = intercept[SparkException] {
- piped.collect()
- }
- assert(exception.getMessage.contains(command.mkString(" ")))
+ assume(testCommandAvailable("cat"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val command = Seq("cat", "nonexistent_file")
+ val piped = nums.pipe(command)
+ val exception = intercept[SparkException] {
+ piped.collect()
}
+ assert(exception.getMessage.contains(command.mkString(" ")))
}
test("basic pipe with separate working directory") {
- if (testCommandAvailable("cat")) {
- val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
- val piped = nums.pipe(Seq("cat"), separateWorkingDir = true)
- val c = piped.collect()
- assert(c.size === 4)
- assert(c(0) === "1")
- assert(c(1) === "2")
- assert(c(2) === "3")
- assert(c(3) === "4")
- val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true)
- val collectPwd = pipedPwd.collect()
- assert(collectPwd(0).contains("tasks/"))
- val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true, bufferSize = 16384).collect()
- // make sure symlinks were created
- assert(pipedLs.length > 0)
- // clean up top level tasks directory
- Utils.deleteRecursively(new File("tasks"))
- } else {
- assert(true)
- }
+ assume(testCommandAvailable("cat"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val piped = nums.pipe(Seq("cat"), separateWorkingDir = true)
+ val c = piped.collect()
+ assert(c.size === 4)
+ assert(c(0) === "1")
+ assert(c(1) === "2")
+ assert(c(2) === "3")
+ assert(c(3) === "4")
+ val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true)
+ val collectPwd = pipedPwd.collect()
+ assert(collectPwd(0).contains("tasks/"))
+ val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true, bufferSize = 16384).collect()
+ // make sure symlinks were created
+ assert(pipedLs.length > 0)
+ // clean up top level tasks directory
+ Utils.deleteRecursively(new File("tasks"))
}
test("test pipe exports map_input_file") {
@@ -219,36 +214,35 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
}
def testExportInputFile(varName: String) {
- if (testCommandAvailable("printenv")) {
- val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable],
- classOf[Text], 2) {
- override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition())
+ assume(testCommandAvailable(envCommand))
+ val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable],
+ classOf[Text], 2) {
+ override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition())
- override val getDependencies = List[Dependency[_]]()
+ override val getDependencies = List[Dependency[_]]()
- override def compute(theSplit: Partition, context: TaskContext) = {
- new InterruptibleIterator[(LongWritable, Text)](context, Iterator((new LongWritable(1),
- new Text("b"))))
- }
+ override def compute(theSplit: Partition, context: TaskContext) = {
+ new InterruptibleIterator[(LongWritable, Text)](context, Iterator((new LongWritable(1),
+ new Text("b"))))
}
- val hadoopPart1 = generateFakeHadoopPartition()
- val pipedRdd =
- new PipedRDD(
- nums,
- PipedRDD.tokenize("printenv " + varName),
- Map(),
- null,
- null,
- false,
- 4092,
- Codec.defaultCharsetCodec.name)
- val tContext = TaskContext.empty()
- val rddIter = pipedRdd.compute(hadoopPart1, tContext)
- val arr = rddIter.toArray
- assert(arr(0) == "/some/path")
- } else {
- // printenv isn't available so just pass the test
}
+ val hadoopPart1 = generateFakeHadoopPartition()
+ val pipedRdd =
+ new PipedRDD(
+ nums,
+ PipedRDD.tokenize(s"$envCommand $varName"),
+ Map(),
+ null,
+ null,
+ false,
+ 4092,
+ Codec.defaultCharsetCodec.name)
+ val tContext = TaskContext.empty()
+ val rddIter = pipedRdd.compute(hadoopPart1, tContext)
+ val arr = rddIter.toArray
+ // On Windows, `cmd.exe /C set` is used which prints out it as `varname=value` format
+ // whereas `printenv` usually prints out `value`. So, `varname=` is stripped here for both.
+ assert(arr(0).stripPrefix(s"$varName=") === "/some/path")
}
def generateFakeHadoopPartition(): HadoopPartition = {
diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
index aa0705987d..acdf21df9a 100644
--- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
@@ -870,19 +870,6 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
verify(endpoint, never()).onDisconnected(any())
verify(endpoint, never()).onNetworkError(any(), any())
}
-
- test("isInRPCThread") {
- val rpcEndpointRef = env.setupEndpoint("isInRPCThread", new RpcEndpoint {
- override val rpcEnv = env
-
- override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case m => context.reply(rpcEnv.isInRPCThread)
- }
- })
- assert(rpcEndpointRef.askWithRetry[Boolean]("hello") === true)
- assert(env.isInRPCThread === false)
- env.stop(rpcEndpointRef)
- }
}
class UnserializableClass
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index b2e7ec5df0..6b314d2ae3 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -17,10 +17,356 @@
package org.apache.spark.scheduler
-import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.mockito.Mockito.when
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark._
import org.apache.spark.internal.config
+import org.apache.spark.util.ManualClock
+
+class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar
+ with LocalSparkContext {
+
+ private val clock = new ManualClock(0)
+
+ private var blacklist: BlacklistTracker = _
+ private var scheduler: TaskSchedulerImpl = _
+ private var conf: SparkConf = _
+
+ override def beforeEach(): Unit = {
+ conf = new SparkConf().setAppName("test").setMaster("local")
+ .set(config.BLACKLIST_ENABLED.key, "true")
+ scheduler = mockTaskSchedWithConf(conf)
+
+ clock.setTime(0)
+ blacklist = new BlacklistTracker(conf, clock)
+ }
+
+ override def afterEach(): Unit = {
+ if (blacklist != null) {
+ blacklist = null
+ }
+ if (scheduler != null) {
+ scheduler.stop()
+ scheduler = null
+ }
+ super.afterEach()
+ }
+
+ // All executors and hosts used in tests should be in this set, so that [[assertEquivalentToSet]]
+ // works. Its OK if its got extraneous entries
+ val allExecutorAndHostIds = {
+ (('A' to 'Z')++ (1 to 100).map(_.toString))
+ .flatMap{ suffix =>
+ Seq(s"host$suffix", s"host-$suffix")
+ }
+ }.toSet
+
+ /**
+ * Its easier to write our tests as if we could directly look at the sets of nodes & executors in
+ * the blacklist. However the api doesn't expose a set, so this is a simple way to test
+ * something similar, since we know the universe of values that might appear in these sets.
+ */
+ def assertEquivalentToSet(f: String => Boolean, expected: Set[String]): Unit = {
+ allExecutorAndHostIds.foreach { id =>
+ val actual = f(id)
+ val exp = expected.contains(id)
+ assert(actual === exp, raw"""for string "$id" """)
+ }
+ }
-class BlacklistTrackerSuite extends SparkFunSuite {
+ def mockTaskSchedWithConf(conf: SparkConf): TaskSchedulerImpl = {
+ sc = new SparkContext(conf)
+ val scheduler = mock[TaskSchedulerImpl]
+ when(scheduler.sc).thenReturn(sc)
+ when(scheduler.mapOutputTracker).thenReturn(SparkEnv.get.mapOutputTracker)
+ scheduler
+ }
+
+ def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = {
+ new TaskSetBlacklist(conf, stageId, clock)
+ }
+
+ test("executors can be blacklisted with only a few failures per stage") {
+ // For many different stages, executor 1 fails a task, then executor 2 succeeds the task,
+ // and then the task set is done. Not enough failures to blacklist the executor *within*
+ // any particular taskset, but we still blacklist the executor overall eventually.
+ // Also, we intentionally have a mix of task successes and failures -- there are even some
+ // successes after the executor is blacklisted. The idea here is those tasks get scheduled
+ // before the executor is blacklisted. We might get successes after blacklisting (because the
+ // executor might be flaky but not totally broken). But successes should not unblacklist the
+ // executor.
+ val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC)
+ var failuresSoFar = 0
+ (0 until failuresUntilBlacklisted * 10).foreach { stageId =>
+ val taskSetBlacklist = createTaskSetBlacklist(stageId)
+ if (stageId % 2 == 0) {
+ // fail one task in every other taskset
+ taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ failuresSoFar += 1
+ }
+ blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures)
+ assert(failuresSoFar == stageId / 2 + 1)
+ if (failuresSoFar < failuresUntilBlacklisted) {
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ } else {
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
+ }
+ }
+ }
+
+ // If an executor has many task failures, but the task set ends up failing, it shouldn't be
+ // counted against the executor.
+ test("executors aren't blacklisted as a result of tasks in failed task sets") {
+ val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC)
+ // for many different stages, executor 1 fails a task, and then the taskSet fails.
+ (0 until failuresUntilBlacklisted * 10).foreach { stage =>
+ val taskSetBlacklist = createTaskSetBlacklist(stage)
+ taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ }
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ }
+
+ Seq(true, false).foreach { succeedTaskSet =>
+ val label = if (succeedTaskSet) "success" else "failure"
+ test(s"stage blacklist updates correctly on stage $label") {
+ // Within one taskset, an executor fails a few times, so it's blacklisted for the taskset.
+ // But if the taskset fails, we shouldn't blacklist the executor after the stage.
+ val taskSetBlacklist = createTaskSetBlacklist(0)
+ // We trigger enough failures for both the taskset blacklist, and the application blacklist.
+ val numFailures = math.max(conf.get(config.MAX_FAILURES_PER_EXEC),
+ conf.get(config.MAX_FAILURES_PER_EXEC_STAGE))
+ (0 until numFailures).foreach { index =>
+ taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = index)
+ }
+ assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ if (succeedTaskSet) {
+ // The task set succeeded elsewhere, so we should count those failures against our executor,
+ // and it should be blacklisted for the entire application.
+ blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures)
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
+ } else {
+ // The task set failed, so we don't count these failures against the executor for other
+ // stages.
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ }
+ }
+ }
+
+ test("blacklisted executors and nodes get recovered with time") {
+ val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0)
+ // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole
+ // application.
+ (0 until 4).foreach { partition =>
+ taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ }
+ blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures)
+ assert(blacklist.nodeBlacklist() === Set())
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
+
+ val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
+ // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole
+ // application. Since that's the second executor that is blacklisted on the same node, we also
+ // blacklist that node.
+ (0 until 4).foreach { partition =>
+ taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ }
+ blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures)
+ assert(blacklist.nodeBlacklist() === Set("hostA"))
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA"))
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2"))
+
+ // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the
+ // blacklist.
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1)
+ blacklist.applyBlacklistTimeout()
+ assert(blacklist.nodeBlacklist() === Set())
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+
+ // Fail one more task, but executor isn't put back into blacklist since the count of failures
+ // on that executor should have been reset to 0.
+ val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2)
+ taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, taskSetBlacklist2.execToFailures)
+ assert(blacklist.nodeBlacklist() === Set())
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ }
+
+ test("blacklist can handle lost executors") {
+ // The blacklist should still work if an executor is killed completely. We should still
+ // be able to blacklist the entire node.
+ val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0)
+ // Lets say that executor 1 dies completely. We get some task failures, but
+ // the taskset then finishes successfully (elsewhere).
+ (0 until 4).foreach { partition =>
+ taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ }
+ blacklist.handleRemovedExecutor("1")
+ blacklist.updateBlacklistForSuccessfulTaskSet(
+ stageId = 0,
+ stageAttemptId = 0,
+ taskSetBlacklist0.execToFailures)
+ assert(blacklist.isExecutorBlacklisted("1"))
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2)
+
+ // Now another executor gets spun up on that host, but it also dies.
+ val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
+ (0 until 4).foreach { partition =>
+ taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ }
+ blacklist.handleRemovedExecutor("2")
+ blacklist.updateBlacklistForSuccessfulTaskSet(
+ stageId = 1,
+ stageAttemptId = 0,
+ taskSetBlacklist1.execToFailures)
+ // We've now had two bad executors on the hostA, so we should blacklist the entire node.
+ assert(blacklist.isExecutorBlacklisted("1"))
+ assert(blacklist.isExecutorBlacklisted("2"))
+ assert(blacklist.isNodeBlacklisted("hostA"))
+
+ // Advance the clock so that executor 1 should no longer be explicitly blacklisted, but
+ // everything else should still be blacklisted.
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1)
+ blacklist.applyBlacklistTimeout()
+ assert(!blacklist.isExecutorBlacklisted("1"))
+ assert(blacklist.isExecutorBlacklisted("2"))
+ assert(blacklist.isNodeBlacklisted("hostA"))
+ // make sure we don't leak memory
+ assert(!blacklist.executorIdToBlacklistStatus.contains("1"))
+ assert(!blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
+ // Advance the timeout again so now hostA should be removed from the blacklist.
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2)
+ blacklist.applyBlacklistTimeout()
+ assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA"))
+ }
+
+ test("task failures expire with time") {
+ // Verifies that 2 failures within the timeout period cause an executor to be blacklisted, but
+ // if task failures are spaced out by more than the timeout period, the first failure is timed
+ // out, and the executor isn't blacklisted.
+ var stageId = 0
+ def failOneTaskInTaskSet(exec: String): Unit = {
+ val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId)
+ taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0)
+ blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures)
+ stageId += 1
+ }
+ failOneTaskInTaskSet(exec = "1")
+ // We have one sporadic failure on exec 2, but that's it. Later checks ensure that we never
+ // blacklist executor 2 despite this one failure.
+ failOneTaskInTaskSet(exec = "2")
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ assert(blacklist.nextExpiryTime === Long.MaxValue)
+
+ // We advance the clock past the expiry time.
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1)
+ val t0 = clock.getTimeMillis()
+ blacklist.applyBlacklistTimeout()
+ assert(blacklist.nextExpiryTime === Long.MaxValue)
+ failOneTaskInTaskSet(exec = "1")
+
+ // Because the 2nd failure on executor 1 happened past the expiry time, nothing should have been
+ // blacklisted.
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+
+ // Now we add one more failure, within the timeout, and it should be counted.
+ clock.setTime(t0 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1)
+ val t1 = clock.getTimeMillis()
+ failOneTaskInTaskSet(exec = "1")
+ blacklist.applyBlacklistTimeout()
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
+ assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
+
+ // Add failures on executor 3, make sure it gets put on the blacklist.
+ clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1)
+ val t2 = clock.getTimeMillis()
+ failOneTaskInTaskSet(exec = "3")
+ failOneTaskInTaskSet(exec = "3")
+ blacklist.applyBlacklistTimeout()
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3"))
+ assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
+
+ // Now we go past the timeout for executor 1, so it should be dropped from the blacklist.
+ clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1)
+ blacklist.applyBlacklistTimeout()
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3"))
+ assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
+
+ // Make sure that we update correctly when we go from having blacklisted executors to
+ // just having tasks with timeouts.
+ clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1)
+ failOneTaskInTaskSet(exec = "4")
+ blacklist.applyBlacklistTimeout()
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3"))
+ assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
+
+ clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1)
+ blacklist.applyBlacklistTimeout()
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to
+ // avoid wasting time checking for expiry of individual task failures.
+ assert(blacklist.nextExpiryTime === Long.MaxValue)
+ }
+
+ test("task failure timeout works as expected for long-running tasksets") {
+ // This ensures that we don't trigger spurious blacklisting for long tasksets, when the taskset
+ // finishes long after the task failures. We create two tasksets, each with one failure.
+ // Individually they shouldn't cause any blacklisting since there is only one failure.
+ // Furthermore, we space the failures out so far that even when both tasksets have completed,
+ // we still don't trigger any blacklisting.
+ val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
+ val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2)
+ // Taskset1 has one failure immediately
+ taskSetBlacklist1.updateBlacklistForFailedTask("host-1", "1", 0)
+ // Then we have a *long* delay, much longer than the timeout, before any other failures or
+ // taskset completion
+ clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS * 5)
+ // After the long delay, we have one failure on taskset 2, on the same executor
+ taskSetBlacklist2.updateBlacklistForFailedTask("host-1", "1", 0)
+ // Finally, we complete both tasksets. Its important here to complete taskset2 *first*. We
+ // want to make sure that when taskset 1 finishes, even though we've now got two task failures,
+ // we realize that the task failure we just added was well before the timeout.
+ clock.advance(1)
+ blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 2, 0, taskSetBlacklist2.execToFailures)
+ clock.advance(1)
+ blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 1, 0, taskSetBlacklist1.execToFailures)
+
+ // Make sure nothing was blacklisted
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
+ }
+
+ test("only blacklist nodes for the application when enough executors have failed on that " +
+ "specific host") {
+ // we blacklist executors on two different hosts -- make sure that doesn't lead to any
+ // node blacklisting
+ val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0)
+ taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1)
+ blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures)
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
+
+ val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
+ taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0)
+ taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1)
+ blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures)
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2"))
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
+
+ // Finally, blacklist another executor on the same node as the original blacklisted executor,
+ // and make sure this time we *do* blacklist the node.
+ val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0)
+ taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 0)
+ taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 1)
+ blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures)
+ assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3"))
+ assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA"))
+ }
test("blacklist still respects legacy configs") {
val conf = new SparkConf().setMaster("local")
@@ -68,6 +414,8 @@ class BlacklistTrackerSuite extends SparkFunSuite {
config.MAX_TASK_ATTEMPTS_PER_NODE,
config.MAX_FAILURES_PER_EXEC_STAGE,
config.MAX_FAILED_EXEC_PER_NODE_STAGE,
+ config.MAX_FAILURES_PER_EXEC,
+ config.MAX_FAILED_EXEC_PER_NODE,
config.BLACKLIST_TIMEOUT_CONF
).foreach { config =>
conf.set(config.key, "0")
diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 230e2c34d0..4c3d0b1021 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -119,19 +119,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
}
test("Event log name") {
+ val baseDirUri = Utils.resolveURI("/base-dir")
// without compression
- assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath(
- Utils.resolveURI("/base-dir"), "app1", None))
+ assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath(
+ baseDirUri, "app1", None))
// with compression
- assert(s"file:/base-dir/app1.lzf" ===
- EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", None, Some("lzf")))
+ assert(s"${baseDirUri.toString}/app1.lzf" ===
+ EventLoggingListener.getLogPath(baseDirUri, "app1", None, Some("lzf")))
// illegal characters in app ID
- assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" ===
- EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"),
+ assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" ===
+ EventLoggingListener.getLogPath(baseDirUri,
"a fine:mind$dollar{bills}.1", None))
// illegal characters in app ID with compression
- assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" ===
- EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"),
+ assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" ===
+ EventLoggingListener.getLogPath(baseDirUri,
"a fine:mind$dollar{bills}.1", None, Some("lz4")))
}
@@ -289,7 +290,7 @@ object EventLoggingListenerSuite {
val conf = new SparkConf
conf.set("spark.eventLog.enabled", "true")
conf.set("spark.eventLog.testing", "true")
- conf.set("spark.eventLog.dir", logDir.toString)
+ conf.set("spark.eventLog.dir", logDir.toUri.toString)
compressionCodec.foreach { codec =>
conf.set("spark.eventLog.compress", "true")
conf.set("spark.io.compression.codec", codec)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
index a757041299..fe6de2bd98 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
@@ -17,12 +17,20 @@
package org.apache.spark.scheduler
+import java.util.Properties
+
+import org.apache.spark.SparkEnv
import org.apache.spark.TaskContext
+import org.apache.spark.executor.TaskMetrics
class FakeTask(
stageId: Int,
partitionId: Int,
- prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, partitionId) {
+ prefLocs: Seq[TaskLocation] = Nil,
+ serializedTaskMetrics: Array[Byte] =
+ SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array())
+ extends Task[Int](stageId, 0, partitionId, new Properties, serializedTaskMetrics) {
+
override def runTask(context: TaskContext): Int = 0
override def preferredLocations: Seq[TaskLocation] = prefLocs
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
index 83288db92b..8c4e389e86 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -158,10 +158,9 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
0 until rdd.partitions.size, resultHandler, () => Unit)
// It's an error if the job completes successfully even though no committer was authorized,
// so throw an exception if the job was allowed to complete.
- val e = intercept[SparkException] {
+ intercept[TimeoutException] {
ThreadUtils.awaitResult(futureAction, 5 seconds)
}
- assert(e.getCause.isInstanceOf[TimeoutException])
assert(tempDir.list().size === 0)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
index c28aa06623..2ba63da881 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
@@ -28,6 +28,8 @@ import scala.reflect.ClassTag
import org.scalactic.TripleEquals
import org.scalatest.Assertions.AssertionsHelper
+import org.scalatest.concurrent.Eventually._
+import org.scalatest.time.SpanSugar._
import org.apache.spark._
import org.apache.spark.TaskState._
@@ -157,8 +159,16 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
}
// When a job fails, we terminate before waiting for all the task end events to come in,
// so there might still be a running task set. So we only check these conditions
- // when the job succeeds
- assert(taskScheduler.runningTaskSets.isEmpty)
+ // when the job succeeds.
+ // When the final task of a taskset completes, we post
+ // the event to the DAGScheduler event loop before we finish processing in the taskscheduler
+ // thread. It's possible the DAGScheduler thread processes the event, finishes the job,
+ // and notifies the job waiter before our original thread in the task scheduler finishes
+ // handling the event and marks the taskset as complete. So its ok if we need to wait a
+ // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race.
+ eventually(timeout(1 second), interval(10 millis)) {
+ assert(taskScheduler.runningTaskSets.isEmpty)
+ }
assert(!backend.hasTasks)
} else {
assert(failure != null)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 9eda79ace1..7004128308 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -62,7 +62,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
val func = (c: TaskContext, i: Iterator[String]) => i.next()
val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func))))
val task = new ResultTask[String, String](
- 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, new TaskMetrics)
+ 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties,
+ closureSerializer.serialize(TaskMetrics.registered).array())
intercept[RuntimeException] {
task.run(0, 0, null)
}
@@ -83,7 +84,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
val func = (c: TaskContext, i: Iterator[String]) => i.next()
val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func))))
val task = new ResultTask[String, String](
- 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, new TaskMetrics)
+ 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties,
+ closureSerializer.serialize(TaskMetrics.registered).array())
intercept[RuntimeException] {
task.run(0, 0, null)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index ee95e4ff7d..c9e682f53c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -171,7 +171,7 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local
val tempDir = Utils.createTempDir()
val srcDir = new File(tempDir, "repro/")
srcDir.mkdirs()
- val excSource = new JavaSourceFromString(new File(srcDir, "MyException").getAbsolutePath,
+ val excSource = new JavaSourceFromString(new File(srcDir, "MyException").toURI.getPath,
"""package repro;
|
|public class MyException extends Exception {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
index a0b6268331..304dc9d47e 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
@@ -21,14 +21,15 @@ import java.nio.ByteBuffer
import scala.collection.mutable.HashMap
-import org.mockito.Matchers.{anyInt, anyString, eq => meq}
-import org.mockito.Mockito.{atLeast, atMost, never, spy, verify, when}
+import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq}
+import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when}
import org.scalatest.BeforeAndAfterEach
import org.scalatest.mock.MockitoSugar
import org.apache.spark._
import org.apache.spark.internal.config
import org.apache.spark.internal.Logging
+import org.apache.spark.storage.BlockManagerId
class FakeSchedulerBackend extends SchedulerBackend {
def start() {}
@@ -44,6 +45,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
var failedTaskSetReason: String = null
var failedTaskSet = false
+ var blacklist: BlacklistTracker = null
var taskScheduler: TaskSchedulerImpl = null
var dagScheduler: DAGScheduler = null
@@ -82,11 +84,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = {
+ blacklist = mock[BlacklistTracker]
val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite")
conf.set(config.BLACKLIST_ENABLED, true)
sc = new SparkContext(conf)
taskScheduler =
- new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4)) {
+ new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), Some(blacklist)) {
override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = {
val tsm = super.createTaskSetManager(taskSet, maxFailures)
// we need to create a spied tsm just so we can set the TaskSetBlacklist
@@ -408,6 +411,95 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
assert(tsm.isZombie)
}
+
+ // the tasksSets complete, so the tracker should be notified of the successful ones
+ verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet(
+ stageId = 0,
+ stageAttemptId = 0,
+ failuresByExec = stageToMockTaskSetBlacklist(0).execToFailures)
+ verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet(
+ stageId = 1,
+ stageAttemptId = 0,
+ failuresByExec = stageToMockTaskSetBlacklist(1).execToFailures)
+ // but we shouldn't update for the failed taskset
+ verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(
+ stageId = meq(2),
+ stageAttemptId = anyInt(),
+ failuresByExec = anyObject())
+ }
+
+ test("scheduled tasks obey node and executor blacklists") {
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
+ (0 to 2).foreach { stageId =>
+ val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0)
+ taskScheduler.submitTasks(taskSet)
+ }
+
+ val offers = IndexedSeq(
+ new WorkerOffer("executor0", "host0", 1),
+ new WorkerOffer("executor1", "host1", 1),
+ new WorkerOffer("executor2", "host1", 1),
+ new WorkerOffer("executor3", "host2", 10),
+ new WorkerOffer("executor4", "host3", 1)
+ )
+
+ // setup our mock blacklist:
+ // host1, executor0 & executor3 are completely blacklisted
+ // This covers everything *except* one core on executor4 / host3, so that everything is still
+ // schedulable.
+ when(blacklist.isNodeBlacklisted("host1")).thenReturn(true)
+ when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true)
+ when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true)
+
+ val stageToTsm = (0 to 2).map { stageId =>
+ val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get
+ stageId -> tsm
+ }.toMap
+
+ val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten
+ firstTaskAttempts.foreach { task => logInfo(s"scheduled $task on ${task.executorId}") }
+ assert(firstTaskAttempts.size === 1)
+ assert(firstTaskAttempts.head.executorId === "executor4")
+ ('0' until '2').foreach { hostNum =>
+ verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum)
+ }
+ }
+
+ test("abort stage when all executors are blacklisted") {
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
+ val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0)
+ taskScheduler.submitTasks(taskSet)
+ val tsm = stageToMockTaskSetManager(0)
+
+ // first just submit some offers so the scheduler knows about all the executors
+ taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 2),
+ WorkerOffer("executor1", "host0", 2),
+ WorkerOffer("executor2", "host0", 2),
+ WorkerOffer("executor3", "host1", 2)
+ ))
+
+ // now say our blacklist updates to blacklist a bunch of resources, but *not* everything
+ when(blacklist.isNodeBlacklisted("host1")).thenReturn(true)
+ when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true)
+
+ // make an offer on the blacklisted resources. We won't schedule anything, but also won't
+ // abort yet, since we know of other resources that work
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 2),
+ WorkerOffer("executor3", "host1", 2)
+ )).flatten.size === 0)
+ assert(!tsm.isZombie)
+
+ // now update the blacklist so that everything really is blacklisted
+ when(blacklist.isExecutorBlacklisted("executor1")).thenReturn(true)
+ when(blacklist.isExecutorBlacklisted("executor2")).thenReturn(true)
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 2),
+ WorkerOffer("executor3", "host1", 2)
+ )).flatten.size === 0)
+ assert(tsm.isZombie)
+ verify(tsm).abort(anyString(), anyObject())
}
/**
@@ -650,6 +742,17 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3")))
}
+ test("scheduler checks for executors that can be expired from blacklist") {
+ taskScheduler = setupScheduler()
+
+ taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0))
+ taskScheduler.resourceOffers(IndexedSeq(
+ new WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ verify(blacklist).applyBlacklistTimeout()
+ }
+
test("if an executor is lost then the state for its running tasks is cleaned up (SPARK-18553)") {
sc = new SparkContext("local", "TaskSchedulerImplSuite")
val taskScheduler = new TaskSchedulerImpl(sc)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
index 8c902af568..6b52c10b2c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
@@ -85,9 +85,9 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
Seq("exec1", "exec2").foreach { exec =>
assert(
- execToFailures(exec).taskToFailureCount === Map(
- 0 -> 1,
- 1 -> 1
+ execToFailures(exec).taskToFailureCountAndFailureTime === Map(
+ 0 -> (1, 0),
+ 1 -> (1, 0)
)
)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index abc8fff30e..2f5b029a96 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -183,7 +183,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
val accumUpdates = taskSet.tasks.head.metrics.internalAccums
// Offer a host with NO_PREF as the constraint,
@@ -236,7 +236,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2"))
val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// An executor that is not NODE_LOCAL should be rejected.
assert(manager.resourceOffer("execC", "host2", ANY) === None)
@@ -257,7 +257,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq() // Last task has no locality prefs
)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None)
@@ -286,7 +286,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq() // Last task has no locality prefs
)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0)
assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1)
@@ -306,7 +306,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host2"))
)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
@@ -344,7 +344,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host3"))
)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
@@ -376,7 +376,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
@@ -393,7 +393,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted
// after the last failure.
@@ -426,7 +426,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
// affinity to exec1 on host1 - which we will fail.
val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, 4, clock)
+ // We don't directly use the application blacklist, but its presence triggers blacklisting
+ // within the taskset.
+ val blacklistTrackerOpt = Some(new BlacklistTracker(conf, clock))
+ val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock)
{
val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)
@@ -515,7 +518,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host2", "execC")),
Seq())
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// Only ANY is valid
assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY)))
// Add a new executor
@@ -546,7 +549,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host1", "execB")),
Seq(TaskLocation("host2", "execC")),
Seq())
- val manager = new TaskSetManager(sched, taskSet, 1, new ManualClock)
+ val manager = new TaskSetManager(sched, taskSet, 1, clock = new ManualClock)
sched.addExecutor("execA", "host1")
manager.executorAdded()
sched.addExecutor("execC", "host2")
@@ -579,7 +582,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host1", "execA")),
Seq(TaskLocation("host1", "execA")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY)))
// Set allowed locality to ANY
@@ -670,7 +673,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(),
Seq(TaskLocation("host3", "execC")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0)
assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None)
@@ -698,7 +701,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(),
Seq(TaskLocation("host3")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// node-local tasks are scheduled without delay
assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0)
@@ -720,7 +723,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(ExecutorCacheTaskLocation("host1", "execA")),
Seq(ExecutorCacheTaskLocation("host2", "execB")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// process-local tasks are scheduled first
assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2)
@@ -740,7 +743,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(ExecutorCacheTaskLocation("host1", "execA")),
Seq(ExecutorCacheTaskLocation("host2", "execB")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// process-local tasks are scheduled first
assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1)
@@ -760,7 +763,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host1", "execA")),
Seq(TaskLocation("host2", "execB.1")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
// Only ANY is valid
assert(manager.myLocalityLevels.sameElements(Array(ANY)))
// Add a new executor
@@ -794,7 +797,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
Seq(TaskLocation("host2")),
Seq(TaskLocation("hdfs_cache_host3")))
val clock = new ManualClock
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY)))
sched.removeExecutor("execA")
manager.executorAdded()
@@ -822,7 +825,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
// Set the speculation multiplier to be 0 so speculative tasks are launched immediately
sc.conf.set("spark.speculation.multiplier", "0.0")
val clock = new ManualClock()
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
task.metrics.internalAccums
}
@@ -876,7 +879,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sc.conf.set("spark.speculation.multiplier", "0.0")
sc.conf.set("spark.speculation.quantile", "0.6")
val clock = new ManualClock()
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
task.metrics.internalAccums
}
@@ -980,17 +983,17 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
sc = new SparkContext("local", "test")
sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
- val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, new ManualClock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock)
assert(manager.name === "TaskSet_0.0")
// Make sure a task set with the same stage ID but different attempt ID has a unique name
val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 1)
- val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, new ManualClock)
+ val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock)
assert(manager2.name === "TaskSet_0.1")
// Make sure a task set with the same attempt ID but different stage ID also has a unique name
val taskSet3 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 1)
- val manager3 = new TaskSetManager(sched, taskSet3, MAX_TASK_FAILURES, new ManualClock)
+ val manager3 = new TaskSetManager(sched, taskSet3, MAX_TASK_FAILURES, clock = new ManualClock)
assert(manager3.name === "TaskSet_1.1")
}
diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
index a61ec74c7d..0f3a4a0361 100644
--- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
@@ -24,6 +24,7 @@ import com.google.common.io.ByteStreams
import org.apache.spark._
import org.apache.spark.internal.config._
+import org.apache.spark.network.util.CryptoUtils
import org.apache.spark.security.CryptoStreamUtils._
import org.apache.spark.serializer.{JavaSerializer, SerializerManager}
import org.apache.spark.storage.TempShuffleBlockId
@@ -33,11 +34,11 @@ class CryptoStreamUtilsSuite extends SparkFunSuite {
test("crypto configuration conversion") {
val sparkKey1 = s"${SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX}a.b.c"
val sparkVal1 = "val1"
- val cryptoKey1 = s"${COMMONS_CRYPTO_CONF_PREFIX}a.b.c"
+ val cryptoKey1 = s"${CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX}a.b.c"
val sparkKey2 = SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX.stripSuffix(".") + "A.b.c"
val sparkVal2 = "val2"
- val cryptoKey2 = s"${COMMONS_CRYPTO_CONF_PREFIX}A.b.c"
+ val cryptoKey2 = s"${CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX}A.b.c"
val conf = new SparkConf()
conf.set(sparkKey1, sparkVal1)
conf.set(sparkKey2, sparkVal2)
diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
index e3ec99685f..e56e440380 100644
--- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.storage
-import java.io.InputStream
+import java.io.{File, InputStream, IOException}
import java.util.concurrent.Semaphore
import scala.concurrent.ExecutionContext.Implicits.global
@@ -31,8 +31,9 @@ import org.scalatest.PrivateMethodTester
import org.apache.spark.{SparkFunSuite, TaskContext}
import org.apache.spark.network._
-import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.shuffle.BlockFetchingListener
+import org.apache.spark.network.util.LimitedInputStream
import org.apache.spark.shuffle.FetchFailedException
@@ -63,7 +64,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
// Create a mock managed buffer for testing
def createMockManagedBuffer(): ManagedBuffer = {
val mockManagedBuffer = mock(classOf[ManagedBuffer])
- when(mockManagedBuffer.createInputStream()).thenReturn(mock(classOf[InputStream]))
+ val in = mock(classOf[InputStream])
+ when(in.read(any())).thenReturn(1)
+ when(in.read(any(), any(), any())).thenReturn(1)
+ when(mockManagedBuffer.createInputStream()).thenReturn(in)
mockManagedBuffer
}
@@ -99,8 +103,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
transfer,
blockManager,
blocksByAddress,
+ (_, in) => in,
48 * 1024 * 1024,
- Int.MaxValue)
+ Int.MaxValue,
+ true)
// 3 local blocks fetched in initialization
verify(blockManager, times(3)).getBlockData(any())
@@ -172,8 +178,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
transfer,
blockManager,
blocksByAddress,
+ (_, in) => in,
48 * 1024 * 1024,
- Int.MaxValue)
+ Int.MaxValue,
+ true)
verify(blocks(ShuffleBlockId(0, 0, 0)), times(0)).release()
iterator.next()._2.close() // close() first block's input stream
@@ -201,9 +209,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
// Make sure remote blocks would return
val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
val blocks = Map[BlockId, ManagedBuffer](
- ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]),
- ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]),
- ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer])
+ ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer()
)
// Semaphore to coordinate event sequence in two different threads.
@@ -235,8 +243,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
transfer,
blockManager,
blocksByAddress,
+ (_, in) => in,
48 * 1024 * 1024,
- Int.MaxValue)
+ Int.MaxValue,
+ true)
// Continue only after the mock calls onBlockFetchFailure
sem.acquire()
@@ -247,4 +257,148 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
intercept[FetchFailedException] { iterator.next() }
intercept[FetchFailedException] { iterator.next() }
}
+
+ test("retry corrupt blocks") {
+ val blockManager = mock(classOf[BlockManager])
+ val localBmId = BlockManagerId("test-client", "test-client", 1)
+ doReturn(localBmId).when(blockManager).blockManagerId
+
+ // Make sure remote blocks would return
+ val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
+ val blocks = Map[BlockId, ManagedBuffer](
+ ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer()
+ )
+
+ // Semaphore to coordinate event sequence in two different threads.
+ val sem = new Semaphore(0)
+
+ val corruptStream = mock(classOf[InputStream])
+ when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt"))
+ val corruptBuffer = mock(classOf[ManagedBuffer])
+ when(corruptBuffer.createInputStream()).thenReturn(corruptStream)
+ val corruptLocalBuffer = new FileSegmentManagedBuffer(null, new File("a"), 0, 100)
+
+ val transfer = mock(classOf[BlockTransferService])
+ when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] {
+ override def answer(invocation: InvocationOnMock): Unit = {
+ val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
+ Future {
+ // Return the first block, and then fail.
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0)))
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 1, 0).toString, corruptBuffer)
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 2, 0).toString, corruptLocalBuffer)
+ sem.release()
+ }
+ }
+ })
+
+ val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
+ (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq))
+
+ val taskContext = TaskContext.empty()
+ val iterator = new ShuffleBlockFetcherIterator(
+ taskContext,
+ transfer,
+ blockManager,
+ blocksByAddress,
+ (_, in) => new LimitedInputStream(in, 100),
+ 48 * 1024 * 1024,
+ Int.MaxValue,
+ true)
+
+ // Continue only after the mock calls onBlockFetchFailure
+ sem.acquire()
+
+ // The first block should be returned without an exception
+ val (id1, _) = iterator.next()
+ assert(id1 === ShuffleBlockId(0, 0, 0))
+
+ when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] {
+ override def answer(invocation: InvocationOnMock): Unit = {
+ val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
+ Future {
+ // Return the first block, and then fail.
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 1, 0).toString, corruptBuffer)
+ sem.release()
+ }
+ }
+ })
+
+ // The next block is corrupt local block (the second one is corrupt and retried)
+ intercept[FetchFailedException] { iterator.next() }
+
+ sem.acquire()
+ intercept[FetchFailedException] { iterator.next() }
+ }
+
+ test("retry corrupt blocks (disabled)") {
+ val blockManager = mock(classOf[BlockManager])
+ val localBmId = BlockManagerId("test-client", "test-client", 1)
+ doReturn(localBmId).when(blockManager).blockManagerId
+
+ // Make sure remote blocks would return
+ val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
+ val blocks = Map[BlockId, ManagedBuffer](
+ ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
+ ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer()
+ )
+
+ // Semaphore to coordinate event sequence in two different threads.
+ val sem = new Semaphore(0)
+
+ val corruptStream = mock(classOf[InputStream])
+ when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt"))
+ val corruptBuffer = mock(classOf[ManagedBuffer])
+ when(corruptBuffer.createInputStream()).thenReturn(corruptStream)
+
+ val transfer = mock(classOf[BlockTransferService])
+ when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] {
+ override def answer(invocation: InvocationOnMock): Unit = {
+ val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
+ Future {
+ // Return the first block, and then fail.
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0)))
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 1, 0).toString, corruptBuffer)
+ listener.onBlockFetchSuccess(
+ ShuffleBlockId(0, 2, 0).toString, corruptBuffer)
+ sem.release()
+ }
+ }
+ })
+
+ val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
+ (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq))
+
+ val taskContext = TaskContext.empty()
+ val iterator = new ShuffleBlockFetcherIterator(
+ taskContext,
+ transfer,
+ blockManager,
+ blocksByAddress,
+ (_, in) => new LimitedInputStream(in, 100),
+ 48 * 1024 * 1024,
+ Int.MaxValue,
+ false)
+
+ // Continue only after the mock calls onBlockFetchFailure
+ sem.acquire()
+
+ // The first block should be returned without an exception
+ val (id1, _) = iterator.next()
+ assert(id1 === ShuffleBlockId(0, 0, 0))
+ val (id2, _) = iterator.next()
+ assert(id2 === ShuffleBlockId(0, 1, 0))
+ val (id3, _) = iterator.next()
+ assert(id3 === ShuffleBlockId(0, 2, 0))
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala
index 667a4db6f7..55c5dd5e24 100644
--- a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala
@@ -44,6 +44,19 @@ class SamplingUtilsSuite extends SparkFunSuite {
assert(sample3.length === 10)
}
+ test("SPARK-18678 reservoirSampleAndCount with tiny input") {
+ val input = Seq(0, 1)
+ val counts = new Array[Int](input.size)
+ for (i <- 0 until 500) {
+ val (samples, inputSize) = SamplingUtils.reservoirSampleAndCount(input.iterator, 1)
+ assert(inputSize === 2)
+ assert(samples.length === 1)
+ counts(samples.head) += 1
+ }
+ // If correct, should be true with prob ~ 0.99999707
+ assert(math.abs(counts(0) - counts(1)) <= 100)
+ }
+
test("computeFraction") {
// test that the computed fraction guarantees enough data points
// in the sample with a failure rate <= 0.0001
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index a3efddeaa5..6be1c72bc6 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -102,3 +102,4 @@ org.apache.spark.scheduler.ExternalClusterManager
.Rbuildignore
org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
spark-warehouse
+structured-streaming/*
diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh
index aa42750f26..b08577c47c 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -150,7 +150,7 @@ if [[ "$1" == "package" ]]; then
NAME=$1
FLAGS=$2
ZINC_PORT=$3
- BUILD_PIP_PACKAGE=$4
+ BUILD_PACKAGE=$4
cp -r spark spark-$SPARK_VERSION-bin-$NAME
cd spark-$SPARK_VERSION-bin-$NAME
@@ -172,11 +172,30 @@ if [[ "$1" == "package" ]]; then
MVN_HOME=`$MVN -version 2>&1 | grep 'Maven home' | awk '{print $NF}'`
- if [ -z "$BUILD_PIP_PACKAGE" ]; then
- echo "Creating distribution without PIP package"
+ if [ -z "$BUILD_PACKAGE" ]; then
+ echo "Creating distribution without PIP/R package"
./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz $FLAGS \
-DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log
cd ..
+ elif [[ "$BUILD_PACKAGE" == "withr" ]]; then
+ echo "Creating distribution with R package"
+ ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --r $FLAGS \
+ -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log
+ cd ..
+
+ echo "Copying and signing R source package"
+ R_DIST_NAME=SparkR_$SPARK_VERSION.tar.gz
+ cp spark-$SPARK_VERSION-bin-$NAME/R/$R_DIST_NAME .
+
+ echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
+ --output $R_DIST_NAME.asc \
+ --detach-sig $R_DIST_NAME
+ echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
+ MD5 $R_DIST_NAME > \
+ $R_DIST_NAME.md5
+ echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
+ SHA512 $R_DIST_NAME > \
+ $R_DIST_NAME.sha
else
echo "Creating distribution with PIP package"
./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --pip $FLAGS \
@@ -219,7 +238,7 @@ if [[ "$1" == "package" ]]; then
FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos"
make_binary_release "hadoop2.3" "-Phadoop-2.3 $FLAGS" "3033" &
make_binary_release "hadoop2.4" "-Phadoop-2.4 $FLAGS" "3034" &
- make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" &
+ make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" "withr" &
make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" "withpip" &
make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn -Pmesos" "3037" &
make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" &
@@ -232,6 +251,8 @@ if [[ "$1" == "package" ]]; then
# Put to new directory:
LFTP mkdir -p $dest_dir
LFTP mput -O $dest_dir 'spark-*'
+ LFTP mput -O $dest_dir 'pyspark-*'
+ LFTP mput -O $dest_dir 'SparkR_*'
# Delete /latest directory and rename new upload to /latest
LFTP "rm -r -f $REMOTE_PARENT_DIR/latest || exit 0"
LFTP mv $dest_dir "$REMOTE_PARENT_DIR/latest"
@@ -239,6 +260,7 @@ if [[ "$1" == "package" ]]; then
LFTP mkdir -p $dest_dir
LFTP mput -O $dest_dir 'spark-*'
LFTP mput -O $dest_dir 'pyspark-*'
+ LFTP mput -O $dest_dir 'SparkR_*'
exit 0
fi
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index 89bfcef4d9..9cbab3d895 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -122,13 +122,13 @@ metrics-graphite-3.1.2.jar
metrics-json-3.1.2.jar
metrics-jvm-3.1.2.jar
minlog-1.3.0.jar
-netty-3.8.0.Final.jar
+netty-3.9.9.Final.jar
netty-all-4.0.42.Final.jar
objenesis-2.1.jar
opencsv-2.3.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
-paranamer-2.3.jar
+paranamer-2.6.jar
parquet-column-1.8.1.jar
parquet-common-1.8.1.jar
parquet-encoding-1.8.1.jar
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index 8df3858825..63ce6c66fd 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -129,13 +129,13 @@ metrics-json-3.1.2.jar
metrics-jvm-3.1.2.jar
minlog-1.3.0.jar
mx4j-3.0.2.jar
-netty-3.8.0.Final.jar
+netty-3.9.9.Final.jar
netty-all-4.0.42.Final.jar
objenesis-2.1.jar
opencsv-2.3.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
-paranamer-2.3.jar
+paranamer-2.6.jar
parquet-column-1.8.1.jar
parquet-common-1.8.1.jar
parquet-encoding-1.8.1.jar
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index 71e7fb6dd2..122d5c27d0 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -129,13 +129,13 @@ metrics-json-3.1.2.jar
metrics-jvm-3.1.2.jar
minlog-1.3.0.jar
mx4j-3.0.2.jar
-netty-3.8.0.Final.jar
+netty-3.9.9.Final.jar
netty-all-4.0.42.Final.jar
objenesis-2.1.jar
opencsv-2.3.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
-paranamer-2.3.jar
+paranamer-2.6.jar
parquet-column-1.8.1.jar
parquet-common-1.8.1.jar
parquet-encoding-1.8.1.jar
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index ba31391495..776aabd111 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -137,13 +137,13 @@ metrics-json-3.1.2.jar
metrics-jvm-3.1.2.jar
minlog-1.3.0.jar
mx4j-3.0.2.jar
-netty-3.8.0.Final.jar
+netty-3.9.9.Final.jar
netty-all-4.0.42.Final.jar
objenesis-2.1.jar
opencsv-2.3.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
-paranamer-2.3.jar
+paranamer-2.6.jar
parquet-column-1.8.1.jar
parquet-common-1.8.1.jar
parquet-encoding-1.8.1.jar
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index b129e5a99e..524e824073 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -138,13 +138,13 @@ metrics-json-3.1.2.jar
metrics-jvm-3.1.2.jar
minlog-1.3.0.jar
mx4j-3.0.2.jar
-netty-3.8.0.Final.jar
+netty-3.9.9.Final.jar
netty-all-4.0.42.Final.jar
objenesis-2.1.jar
opencsv-2.3.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
-paranamer-2.3.jar
+paranamer-2.6.jar
parquet-column-1.8.1.jar
parquet-common-1.8.1.jar
parquet-encoding-1.8.1.jar
diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
index 49b46fbc3f..6c5ae0d629 100755
--- a/dev/make-distribution.sh
+++ b/dev/make-distribution.sh
@@ -34,6 +34,7 @@ DISTDIR="$SPARK_HOME/dist"
MAKE_TGZ=false
MAKE_PIP=false
+MAKE_R=false
NAME=none
MVN="$SPARK_HOME/build/mvn"
@@ -41,7 +42,7 @@ function exit_with_usage {
echo "make-distribution.sh - tool for making binary distributions of Spark"
echo ""
echo "usage:"
- cl_options="[--name] [--tgz] [--pip] [--mvn ]"
+ cl_options="[--name] [--tgz] [--pip] [--r] [--mvn ]"
echo "make-distribution.sh $cl_options "
echo "See Spark's \"Building Spark\" doc for correct Maven options."
echo ""
@@ -71,6 +72,9 @@ while (( "$#" )); do
--pip)
MAKE_PIP=true
;;
+ --r)
+ MAKE_R=true
+ ;;
--mvn)
MVN="$2"
shift
@@ -98,6 +102,13 @@ if [ -z "$JAVA_HOME" ]; then
echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm"
fi
fi
+
+ if [ -z "$JAVA_HOME" ]; then
+ if [ `command -v java` ]; then
+ # If java is in /usr/bin/java, we want /usr
+ JAVA_HOME="$(dirname $(dirname $(which java)))"
+ fi
+ fi
fi
if [ -z "$JAVA_HOME" ]; then
@@ -208,11 +219,30 @@ cp -r "$SPARK_HOME/data" "$DISTDIR"
# Make pip package
if [ "$MAKE_PIP" == "true" ]; then
echo "Building python distribution package"
- cd $SPARK_HOME/python
+ pushd "$SPARK_HOME/python" > /dev/null
python setup.py sdist
- cd ..
+ popd > /dev/null
+else
+ echo "Skipping building python distribution package"
+fi
+
+# Make R package - this is used for both CRAN release and packing R layout into distribution
+if [ "$MAKE_R" == "true" ]; then
+ echo "Building R source package"
+ R_PACKAGE_VERSION=`grep Version $SPARK_HOME/R/pkg/DESCRIPTION | awk '{print $NF}'`
+ pushd "$SPARK_HOME/R" > /dev/null
+ # Build source package and run full checks
+ # Install source package to get it to generate vignettes, etc.
+ # Do not source the check-cran.sh - it should be run from where it is for it to set SPARK_HOME
+ NO_TESTS=1 CLEAN_INSTALL=1 "$SPARK_HOME/"R/check-cran.sh
+ # Move R source package to match the Spark release version if the versions are not the same.
+ # NOTE(shivaram): `mv` throws an error on Linux if source and destination are same file
+ if [ "$R_PACKAGE_VERSION" != "$VERSION" ]; then
+ mv $SPARK_HOME/R/SparkR_"$R_PACKAGE_VERSION".tar.gz $SPARK_HOME/R/SparkR_"$VERSION".tar.gz
+ fi
+ popd > /dev/null
else
- echo "Skipping creating pip installable PySpark"
+ echo "Skipping building R source package"
fi
# Copy other things
@@ -221,6 +251,12 @@ cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf
cp "$SPARK_HOME/README.md" "$DISTDIR"
cp -r "$SPARK_HOME/bin" "$DISTDIR"
cp -r "$SPARK_HOME/python" "$DISTDIR"
+
+# Remove the python distribution from dist/ if we built it
+if [ "$MAKE_PIP" == "true" ]; then
+ rm -f $DISTDIR/python/dist/pyspark-*.tar.gz
+fi
+
cp -r "$SPARK_HOME/sbin" "$DISTDIR"
# Copy SparkR if it exists
if [ -d "$SPARK_HOME"/R/lib/SparkR ]; then
diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py
index 1d1e72facc..bb286af763 100755
--- a/dev/run-tests-jenkins.py
+++ b/dev/run-tests-jenkins.py
@@ -80,7 +80,7 @@ def pr_message(build_display_name,
short_commit_hash,
commit_url,
str(' ' + post_msg + '.') if post_msg else '.')
- return '**[Test build %s %s](%sconsoleFull)** for PR %s at commit [`%s`](%s)%s' % str_args
+ return '**[Test build %s %s](%stestReport)** for PR %s at commit [`%s`](%s)%s' % str_args
def run_pr_checks(pr_tests, ghprb_actual_commit, sha1):
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index b34ab51f3b..10ad1fe3aa 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -245,7 +245,8 @@ def __hash__(self):
name="streaming-kafka-0-10",
dependencies=[streaming],
source_file_regexes=[
- "external/kafka-0-10",
+ # The ending "/" is necessary otherwise it will include "sql-kafka" codes
+ "external/kafka-0-10/",
"external/kafka-0-10-assembly",
],
sbt_test_goals=[
@@ -469,7 +470,7 @@ def __hash__(self):
name="yarn",
dependencies=[],
source_file_regexes=[
- "yarn/",
+ "resource-managers/yarn/",
"common/network-yarn/",
],
build_profile_flags=["-Pyarn"],
@@ -485,7 +486,7 @@ def __hash__(self):
mesos = Module(
name="mesos",
dependencies=[],
- source_file_regexes=["mesos/"],
+ source_file_regexes=["resource-managers/mesos/"],
build_profile_flags=["-Pmesos"],
sbt_test_goals=["mesos/test"]
)
diff --git a/docs/README.md b/docs/README.md
index ffd3b5712b..90e10a104b 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -69,4 +69,5 @@ may take some time as it generates all of the scaladoc. The jekyll plugin also
PySpark docs using [Sphinx](http://sphinx-doc.org/).
NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1
-jekyll`.
+jekyll`. In addition, `SKIP_SCALADOC=1`, `SKIP_PYTHONDOC=1`, and `SKIP_RDOC=1` can be used to skip a single
+step of the corresponding language.
diff --git a/docs/_config.yml b/docs/_config.yml
index e4fc093fe7..83bb30598d 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -14,8 +14,8 @@ include:
# These allow the documentation to be updated with newer releases
# of Spark, Scala, and Mesos.
-SPARK_VERSION: 2.1.0-SNAPSHOT
-SPARK_VERSION_SHORT: 2.1.0
+SPARK_VERSION: 2.2.0-SNAPSHOT
+SPARK_VERSION_SHORT: 2.2.0
SCALA_BINARY_VERSION: "2.11"
SCALA_VERSION: "2.11.7"
MESOS_VERSION: 1.0.0
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index f926d67e6b..95e3ba35e9 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -113,33 +113,41 @@
File.open(css_file, 'a') { |f| f.write("\n" + css.join()) }
end
- # Build Sphinx docs for Python
+ if not (ENV['SKIP_PYTHONDOC'] == '1')
+ # Build Sphinx docs for Python
- puts "Moving to python/docs directory and building sphinx."
- cd("../python/docs")
- system("make html") || raise("Python doc generation failed")
+ puts "Moving to python/docs directory and building sphinx."
+ cd("../python/docs")
+ system("make html") || raise("Python doc generation failed")
- puts "Moving back into home dir."
- cd("../../")
+ puts "Moving back into docs dir."
+ cd("../../docs")
+
+ puts "Making directory api/python"
+ mkdir_p "api/python"
+
+ puts "cp -r ../python/docs/_build/html/. api/python"
+ cp_r("../python/docs/_build/html/.", "api/python")
+ end
- puts "Making directory api/python"
- mkdir_p "docs/api/python"
+ if not (ENV['SKIP_RDOC'] == '1')
+ # Build SparkR API docs
- puts "cp -r python/docs/_build/html/. docs/api/python"
- cp_r("python/docs/_build/html/.", "docs/api/python")
+ puts "Moving to R directory and building roxygen docs."
+ cd("../R")
+ system("./create-docs.sh") || raise("R doc generation failed")
- # Build SparkR API docs
- puts "Moving to R directory and building roxygen docs."
- cd("R")
- system("./create-docs.sh") || raise("R doc generation failed")
+ puts "Moving back into docs dir."
+ cd("../docs")
- puts "Moving back into home dir."
- cd("../")
+ puts "Making directory api/R"
+ mkdir_p "api/R"
- puts "Making directory api/R"
- mkdir_p "docs/api/R"
+ puts "cp -r ../R/pkg/html/. api/R"
+ cp_r("../R/pkg/html/.", "api/R")
- puts "cp -r R/pkg/html/. docs/api/R"
- cp_r("R/pkg/html/.", "docs/api/R")
+ puts "cp ../R/pkg/DESCRIPTION api"
+ cp("../R/pkg/DESCRIPTION", "api")
+ end
end
diff --git a/docs/configuration.md b/docs/configuration.md
index d8800e93da..39bfb3a05b 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -699,6 +699,15 @@ Apart from these, the following properties are also available, and may be useful
This is the URL where your proxy is running. This URL is for proxy which is running in front of Spark Master. This is useful when running proxy for authentication e.g. OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to reach your proxy.
|
+