diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 740f12e7d13d4..974e5c5ffd0a0 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -201,7 +201,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP
val stages = jobData.stageIds.map { stageId =>
// This could be empty if the listener hasn't received information about the
// stage or if the stage information has been garbage collected
- store.stageData(stageId).lastOption.getOrElse {
+ store.asOption(store.lastStageAttempt(stageId)).getOrElse {
new v1.StageData(
v1.StageStatus.PENDING,
stageId,
@@ -336,8 +336,14 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP
content ++= makeTimeline(activeStages ++ completedStages ++ failedStages,
store.executorList(false), appStartTime)
- content ++= UIUtils.showDagVizForJob(
- jobId, store.operationGraphForJob(jobId))
+ val operationGraphContent = store.asOption(store.operationGraphForJob(jobId)) match {
+ case Some(operationGraph) => UIUtils.showDagVizForJob(jobId, operationGraph)
+ case None =>
+
+ No DAG visualization information to display for job {jobId}
+
+ }
+ content ++= operationGraphContent
if (shouldShowActiveStages) {
content ++= Active Stages ({activeStages.size}) ++
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index 99eab1b2a27d8..ff1b75e5c5065 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -34,10 +34,10 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore)
val killEnabled = parent.killEnabled
def isFairScheduler: Boolean = {
- store.environmentInfo().sparkProperties.toMap
- .get("spark.scheduler.mode")
- .map { mode => mode == SchedulingMode.FAIR }
- .getOrElse(false)
+ store
+ .environmentInfo()
+ .sparkProperties
+ .contains(("spark.scheduler.mode", SchedulingMode.FAIR.toString))
}
def getSparkUser: String = parent.getSparkUser
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 11a6a34344976..7ab433655233e 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -19,25 +19,23 @@ package org.apache.spark.ui.jobs
import java.net.URLEncoder
import java.util.Date
+import java.util.concurrent.TimeUnit
import javax.servlet.http.HttpServletRequest
import scala.collection.mutable.{HashMap, HashSet}
-import scala.xml.{Elem, Node, Unparsed}
+import scala.xml.{Node, Unparsed}
import org.apache.commons.lang3.StringEscapeUtils
-import org.apache.spark.SparkConf
-import org.apache.spark.internal.config._
import org.apache.spark.scheduler.TaskLocality
-import org.apache.spark.status.AppStatusStore
+import org.apache.spark.status._
import org.apache.spark.status.api.v1._
import org.apache.spark.ui._
-import org.apache.spark.util.{Distribution, Utils}
+import org.apache.spark.util.Utils
/** Page showing statistics and task list for a given stage */
private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends WebUIPage("stage") {
import ApiHelper._
- import StagePage._
private val TIMELINE_LEGEND = {
@@ -67,17 +65,17 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
// if we find that it's okay.
private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000)
- private def getLocalitySummaryString(stageData: StageData, taskList: Seq[TaskData]): String = {
- val localities = taskList.map(_.taskLocality)
- val localityCounts = localities.groupBy(identity).mapValues(_.size)
+ private def getLocalitySummaryString(localitySummary: Map[String, Long]): String = {
val names = Map(
TaskLocality.PROCESS_LOCAL.toString() -> "Process local",
TaskLocality.NODE_LOCAL.toString() -> "Node local",
TaskLocality.RACK_LOCAL.toString() -> "Rack local",
TaskLocality.ANY.toString() -> "Any")
- val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) =>
- s"${names(locality)}: $count"
- }
+ val localityNamesAndCounts = names.flatMap { case (key, name) =>
+ localitySummary.get(key).map { count =>
+ s"$name: $count"
+ }
+ }.toSeq
localityNamesAndCounts.sorted.mkString("; ")
}
@@ -108,7 +106,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)"
val stageData = parent.store
- .asOption(parent.store.stageAttempt(stageId, stageAttemptId, details = true))
+ .asOption(parent.store.stageAttempt(stageId, stageAttemptId, details = false))
.getOrElse {
val content =
@@ -117,8 +115,10 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
return UIUtils.headerSparkPage(stageHeader, content, parent)
}
- val tasks = stageData.tasks.getOrElse(Map.empty).values.toSeq
- if (tasks.isEmpty) {
+ val localitySummary = store.localitySummary(stageData.stageId, stageData.attemptId)
+
+ val totalTasks = taskCount(stageData)
+ if (totalTasks == 0) {
val content =
Summary Metrics No tasks have started yet
@@ -127,18 +127,14 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
return UIUtils.headerSparkPage(stageHeader, content, parent)
}
+ val storedTasks = store.taskCount(stageData.stageId, stageData.attemptId)
val numCompleted = stageData.numCompleteTasks
- val totalTasks = stageData.numActiveTasks + stageData.numCompleteTasks +
- stageData.numFailedTasks + stageData.numKilledTasks
- val totalTasksNumStr = if (totalTasks == tasks.size) {
+ val totalTasksNumStr = if (totalTasks == storedTasks) {
s"$totalTasks"
} else {
- s"$totalTasks, showing ${tasks.size}"
+ s"$storedTasks, showing ${totalTasks}"
}
- val externalAccumulables = stageData.accumulatorUpdates
- val hasAccumulators = externalAccumulables.size > 0
-
val summary =
@@ -148,7 +144,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
-
Locality Level Summary:
- {getLocalitySummaryString(stageData, tasks)}
+ {getLocalitySummaryString(localitySummary)}
{if (hasInput(stageData)) {
-
@@ -261,12 +257,16 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
def accumulableRow(acc: AccumulableInfo): Seq[Node] = {
-
{acc.name} | {acc.value} |
+ if (acc.name != null && acc.value != null) {
+ {acc.name} | {acc.value} |
+ } else {
+ Nil
+ }
}
val accumulableTable = UIUtils.listingTable(
accumulableHeaders,
accumulableRow,
- externalAccumulables.toSeq)
+ stageData.accumulatorUpdates.toSeq)
val page: Int = {
// If the user has changed to a larger page size, then go to page 1 in order to avoid
@@ -280,16 +280,9 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val currentTime = System.currentTimeMillis()
val (taskTable, taskTableHTML) = try {
val _taskTable = new TaskPagedTable(
- parent.conf,
+ stageData,
UIUtils.prependBaseUri(parent.basePath) +
s"/stages/stage?id=${stageId}&attempt=${stageAttemptId}",
- tasks,
- hasAccumulators,
- hasInput(stageData),
- hasOutput(stageData),
- hasShuffleRead(stageData),
- hasShuffleWrite(stageData),
- hasBytesSpilled(stageData),
currentTime,
pageSize = taskPageSize,
sortColumn = taskSortColumn,
@@ -320,217 +313,155 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
| }
|});
""".stripMargin
- }
+ }
}
- val taskIdsInPage = if (taskTable == null) Set.empty[Long]
- else taskTable.dataSource.slicedTaskIds
+ val metricsSummary = store.taskSummary(stageData.stageId, stageData.attemptId,
+ Array(0, 0.25, 0.5, 0.75, 1.0))
- // Excludes tasks which failed and have incomplete metrics
- val validTasks = tasks.filter(t => t.status == "SUCCESS" && t.taskMetrics.isDefined)
-
- val summaryTable: Option[Seq[Node]] =
- if (validTasks.size == 0) {
- None
- } else {
- def getDistributionQuantiles(data: Seq[Double]): IndexedSeq[Double] = {
- Distribution(data).get.getQuantiles()
- }
- def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = {
- getDistributionQuantiles(times).map { millis =>
- {UIUtils.formatDuration(millis.toLong)} |
- }
- }
- def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = {
- getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)} | )
+ val summaryTable = metricsSummary.map { metrics =>
+ def timeQuantiles(data: IndexedSeq[Double]): Seq[Node] = {
+ data.map { millis =>
+ {UIUtils.formatDuration(millis.toLong)} |
}
+ }
- val deserializationTimes = validTasks.map { task =>
- task.taskMetrics.get.executorDeserializeTime.toDouble
- }
- val deserializationQuantiles =
-
-
- Task Deserialization Time
-
- | +: getFormattedTimeQuantiles(deserializationTimes)
-
- val serviceTimes = validTasks.map(_.taskMetrics.get.executorRunTime.toDouble)
- val serviceQuantiles = Duration | +: getFormattedTimeQuantiles(serviceTimes)
-
- val gcTimes = validTasks.map(_.taskMetrics.get.jvmGcTime.toDouble)
- val gcQuantiles =
-
- GC Time
-
- | +: getFormattedTimeQuantiles(gcTimes)
-
- val serializationTimes = validTasks.map(_.taskMetrics.get.resultSerializationTime.toDouble)
- val serializationQuantiles =
-
-
- Result Serialization Time
-
- | +: getFormattedTimeQuantiles(serializationTimes)
-
- val gettingResultTimes = validTasks.map(getGettingResultTime(_, currentTime).toDouble)
- val gettingResultQuantiles =
-
-
- Getting Result Time
-
- | +:
- getFormattedTimeQuantiles(gettingResultTimes)
-
- val peakExecutionMemory = validTasks.map(_.taskMetrics.get.peakExecutionMemory.toDouble)
- val peakExecutionMemoryQuantiles = {
-
-
- Peak Execution Memory
-
- | +: getFormattedSizeQuantiles(peakExecutionMemory)
+ def sizeQuantiles(data: IndexedSeq[Double]): Seq[Node] = {
+ data.map { size =>
+ {Utils.bytesToString(size.toLong)} |
}
+ }
- // The scheduler delay includes the network delay to send the task to the worker
- // machine and to send back the result (but not the time to fetch the task result,
- // if it needed to be fetched from the block manager on the worker).
- val schedulerDelays = validTasks.map { task =>
- getSchedulerDelay(task, task.taskMetrics.get, currentTime).toDouble
- }
- val schedulerDelayTitle = Scheduler Delay |
- val schedulerDelayQuantiles = schedulerDelayTitle +:
- getFormattedTimeQuantiles(schedulerDelays)
- def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double])
- : Seq[Elem] = {
- val recordDist = getDistributionQuantiles(records).iterator
- getDistributionQuantiles(data).map(d =>
- {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} |
- )
+ def sizeQuantilesWithRecords(
+ data: IndexedSeq[Double],
+ records: IndexedSeq[Double]) : Seq[Node] = {
+ data.zip(records).map { case (d, r) =>
+ {s"${Utils.bytesToString(d.toLong)} / ${r.toLong}"} |
}
+ }
- val inputSizes = validTasks.map(_.taskMetrics.get.inputMetrics.bytesRead.toDouble)
- val inputRecords = validTasks.map(_.taskMetrics.get.inputMetrics.recordsRead.toDouble)
- val inputQuantiles = Input Size / Records | +:
- getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords)
+ def titleCell(title: String, tooltip: String): Seq[Node] = {
+
+
+ {title}
+
+ |
+ }
- val outputSizes = validTasks.map(_.taskMetrics.get.outputMetrics.bytesWritten.toDouble)
- val outputRecords = validTasks.map(_.taskMetrics.get.outputMetrics.recordsWritten.toDouble)
- val outputQuantiles = Output Size / Records | +:
- getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords)
+ def simpleTitleCell(title: String): Seq[Node] = {title} |
- val shuffleReadBlockedTimes = validTasks.map { task =>
- task.taskMetrics.get.shuffleReadMetrics.fetchWaitTime.toDouble
- }
- val shuffleReadBlockedQuantiles =
-
-
- Shuffle Read Blocked Time
-
- | +:
- getFormattedTimeQuantiles(shuffleReadBlockedTimes)
-
- val shuffleReadTotalSizes = validTasks.map { task =>
- totalBytesRead(task.taskMetrics.get.shuffleReadMetrics).toDouble
- }
- val shuffleReadTotalRecords = validTasks.map { task =>
- task.taskMetrics.get.shuffleReadMetrics.recordsRead.toDouble
- }
- val shuffleReadTotalQuantiles =
-
-
- Shuffle Read Size / Records
-
- | +:
- getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords)
-
- val shuffleReadRemoteSizes = validTasks.map { task =>
- task.taskMetrics.get.shuffleReadMetrics.remoteBytesRead.toDouble
- }
- val shuffleReadRemoteQuantiles =
-
-
- Shuffle Remote Reads
-
- | +:
- getFormattedSizeQuantiles(shuffleReadRemoteSizes)
-
- val shuffleWriteSizes = validTasks.map { task =>
- task.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toDouble
- }
+ val deserializationQuantiles = titleCell("Task Deserialization Time",
+ ToolTips.TASK_DESERIALIZATION_TIME) ++ timeQuantiles(metrics.executorDeserializeTime)
- val shuffleWriteRecords = validTasks.map { task =>
- task.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toDouble
- }
+ val serviceQuantiles = simpleTitleCell("Duration") ++ timeQuantiles(metrics.executorRunTime)
- val shuffleWriteQuantiles = Shuffle Write Size / Records | +:
- getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords)
+ val gcQuantiles = titleCell("GC Time", ToolTips.GC_TIME) ++ timeQuantiles(metrics.jvmGcTime)
- val memoryBytesSpilledSizes = validTasks.map(_.taskMetrics.get.memoryBytesSpilled.toDouble)
- val memoryBytesSpilledQuantiles = Shuffle spill (memory) | +:
- getFormattedSizeQuantiles(memoryBytesSpilledSizes)
+ val serializationQuantiles = titleCell("Result Serialization Time",
+ ToolTips.RESULT_SERIALIZATION_TIME) ++ timeQuantiles(metrics.resultSerializationTime)
- val diskBytesSpilledSizes = validTasks.map(_.taskMetrics.get.diskBytesSpilled.toDouble)
- val diskBytesSpilledQuantiles = Shuffle spill (disk) | +:
- getFormattedSizeQuantiles(diskBytesSpilledSizes)
+ val gettingResultQuantiles = titleCell("Getting Result Time", ToolTips.GETTING_RESULT_TIME) ++
+ timeQuantiles(metrics.gettingResultTime)
- val listings: Seq[Seq[Node]] = Seq(
- {serviceQuantiles} ,
- {schedulerDelayQuantiles} ,
-
- {deserializationQuantiles}
-
- {gcQuantiles} ,
-
- {serializationQuantiles}
- ,
- {gettingResultQuantiles} ,
-
- {peakExecutionMemoryQuantiles}
- ,
- if (hasInput(stageData)) {inputQuantiles} else Nil,
- if (hasOutput(stageData)) {outputQuantiles} else Nil,
- if (hasShuffleRead(stageData)) {
-
- {shuffleReadBlockedQuantiles}
-
- {shuffleReadTotalQuantiles}
-
- {shuffleReadRemoteQuantiles}
-
- } else {
- Nil
- },
- if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil,
- if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil,
- if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil)
-
- val quantileHeaders = Seq("Metric", "Min", "25th percentile",
- "Median", "75th percentile", "Max")
- // The summary table does not use CSS to stripe rows, which doesn't work with hidden
- // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows).
- Some(UIUtils.listingTable(
- quantileHeaders,
- identity[Seq[Node]],
- listings,
- fixedWidth = true,
- id = Some("task-summary-table"),
- stripeRowsWithCss = false))
+ val peakExecutionMemoryQuantiles = titleCell("Peak Execution Memory",
+ ToolTips.PEAK_EXECUTION_MEMORY) ++ sizeQuantiles(metrics.peakExecutionMemory)
+
+ // The scheduler delay includes the network delay to send the task to the worker
+ // machine and to send back the result (but not the time to fetch the task result,
+ // if it needed to be fetched from the block manager on the worker).
+ val schedulerDelayQuantiles = titleCell("Scheduler Delay", ToolTips.SCHEDULER_DELAY) ++
+ timeQuantiles(metrics.schedulerDelay)
+
+ def inputQuantiles: Seq[Node] = {
+ simpleTitleCell("Input Size / Records") ++
+ sizeQuantilesWithRecords(metrics.inputMetrics.bytesRead, metrics.inputMetrics.recordsRead)
+ }
+
+ def outputQuantiles: Seq[Node] = {
+ simpleTitleCell("Output Size / Records") ++
+ sizeQuantilesWithRecords(metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten)
+ }
+
+ def shuffleReadBlockedQuantiles: Seq[Node] = {
+ titleCell("Shuffle Read Blocked Time", ToolTips.SHUFFLE_READ_BLOCKED_TIME) ++
+ timeQuantiles(metrics.shuffleReadMetrics.fetchWaitTime)
+ }
+
+ def shuffleReadTotalQuantiles: Seq[Node] = {
+ titleCell("Shuffle Read Size / Records", ToolTips.SHUFFLE_READ) ++
+ sizeQuantilesWithRecords(metrics.shuffleReadMetrics.readBytes,
+ metrics.shuffleReadMetrics.readRecords)
+ }
+
+ def shuffleReadRemoteQuantiles: Seq[Node] = {
+ titleCell("Shuffle Remote Reads", ToolTips.SHUFFLE_READ_REMOTE_SIZE) ++
+ sizeQuantiles(metrics.shuffleReadMetrics.remoteBytesRead)
+ }
+
+ def shuffleWriteQuantiles: Seq[Node] = {
+ simpleTitleCell("Shuffle Write Size / Records") ++
+ sizeQuantilesWithRecords(metrics.shuffleWriteMetrics.writeBytes,
+ metrics.shuffleWriteMetrics.writeRecords)
+ }
+
+ def memoryBytesSpilledQuantiles: Seq[Node] = {
+ simpleTitleCell("Shuffle spill (memory)") ++ sizeQuantiles(metrics.memoryBytesSpilled)
+ }
+
+ def diskBytesSpilledQuantiles: Seq[Node] = {
+ simpleTitleCell("Shuffle spill (disk)") ++ sizeQuantiles(metrics.diskBytesSpilled)
}
+ val listings: Seq[Seq[Node]] = Seq(
+ {serviceQuantiles} ,
+ {schedulerDelayQuantiles} ,
+
+ {deserializationQuantiles}
+
+ {gcQuantiles} ,
+
+ {serializationQuantiles}
+ ,
+ {gettingResultQuantiles} ,
+
+ {peakExecutionMemoryQuantiles}
+ ,
+ if (hasInput(stageData)) {inputQuantiles} else Nil,
+ if (hasOutput(stageData)) {outputQuantiles} else Nil,
+ if (hasShuffleRead(stageData)) {
+
+ {shuffleReadBlockedQuantiles}
+
+ {shuffleReadTotalQuantiles}
+
+ {shuffleReadRemoteQuantiles}
+
+ } else {
+ Nil
+ },
+ if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil,
+ if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil,
+ if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil)
+
+ val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile",
+ "Max")
+ // The summary table does not use CSS to stripe rows, which doesn't work with hidden
+ // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows).
+ UIUtils.listingTable(
+ quantileHeaders,
+ identity[Seq[Node]],
+ listings,
+ fixedWidth = true,
+ id = Some("task-summary-table"),
+ stripeRowsWithCss = false)
+ }
+
val executorTable = new ExecutorTable(stageData, parent.store)
val maybeAccumulableTable: Seq[Node] =
- if (hasAccumulators) { Accumulators ++ accumulableTable } else Seq()
+ if (hasAccumulators(stageData)) { Accumulators ++ accumulableTable } else Seq()
val aggMetrics =
taskIdsInPage.contains(t.taskId) },
+ Option(taskTable).map(_.dataSource.tasks).getOrElse(Nil),
currentTime) ++
++
{summaryTable.getOrElse("No tasks have reported metrics yet.")} ++
@@ -593,10 +524,9 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val serializationTimeProportion = toProportion(serializationTime)
val deserializationTime = metricsOpt.map(_.executorDeserializeTime).getOrElse(0L)
val deserializationTimeProportion = toProportion(deserializationTime)
- val gettingResultTime = getGettingResultTime(taskInfo, currentTime)
+ val gettingResultTime = AppStatusUtils.gettingResultTime(taskInfo)
val gettingResultTimeProportion = toProportion(gettingResultTime)
- val schedulerDelay =
- metricsOpt.map(getSchedulerDelay(taskInfo, _, currentTime)).getOrElse(0L)
+ val schedulerDelay = AppStatusUtils.schedulerDelay(taskInfo)
val schedulerDelayProportion = toProportion(schedulerDelay)
val executorOverhead = serializationTime + deserializationTime
@@ -708,7 +638,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
{
if (MAX_TIMELINE_TASKS < tasks.size) {
- This stage has more than the maximum number of tasks that can be shown in the
+ This page has more than the maximum number of tasks that can be shown in the
visualization! Only the most recent {MAX_TIMELINE_TASKS} tasks
(of {tasks.size} total) are shown.
@@ -733,402 +663,49 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
}
-private[ui] object StagePage {
- private[ui] def getGettingResultTime(info: TaskData, currentTime: Long): Long = {
- info.resultFetchStart match {
- case Some(start) =>
- info.duration match {
- case Some(duration) =>
- info.launchTime.getTime() + duration - start.getTime()
-
- case _ =>
- currentTime - start.getTime()
- }
-
- case _ =>
- 0L
- }
- }
-
- private[ui] def getSchedulerDelay(
- info: TaskData,
- metrics: TaskMetrics,
- currentTime: Long): Long = {
- info.duration match {
- case Some(duration) =>
- val executorOverhead = metrics.executorDeserializeTime + metrics.resultSerializationTime
- math.max(
- 0,
- duration - metrics.executorRunTime - executorOverhead -
- getGettingResultTime(info, currentTime))
-
- case _ =>
- // The task is still running and the metrics like executorRunTime are not available.
- 0L
- }
- }
-
-}
-
-private[ui] case class TaskTableRowInputData(inputSortable: Long, inputReadable: String)
-
-private[ui] case class TaskTableRowOutputData(outputSortable: Long, outputReadable: String)
-
-private[ui] case class TaskTableRowShuffleReadData(
- shuffleReadBlockedTimeSortable: Long,
- shuffleReadBlockedTimeReadable: String,
- shuffleReadSortable: Long,
- shuffleReadReadable: String,
- shuffleReadRemoteSortable: Long,
- shuffleReadRemoteReadable: String)
-
-private[ui] case class TaskTableRowShuffleWriteData(
- writeTimeSortable: Long,
- writeTimeReadable: String,
- shuffleWriteSortable: Long,
- shuffleWriteReadable: String)
-
-private[ui] case class TaskTableRowBytesSpilledData(
- memoryBytesSpilledSortable: Long,
- memoryBytesSpilledReadable: String,
- diskBytesSpilledSortable: Long,
- diskBytesSpilledReadable: String)
-
-/**
- * Contains all data that needs for sorting and generating HTML. Using this one rather than
- * TaskData to avoid creating duplicate contents during sorting the data.
- */
-private[ui] class TaskTableRowData(
- val index: Int,
- val taskId: Long,
- val attempt: Int,
- val speculative: Boolean,
- val status: String,
- val taskLocality: String,
- val executorId: String,
- val host: String,
- val launchTime: Long,
- val duration: Long,
- val formatDuration: String,
- val schedulerDelay: Long,
- val taskDeserializationTime: Long,
- val gcTime: Long,
- val serializationTime: Long,
- val gettingResultTime: Long,
- val peakExecutionMemoryUsed: Long,
- val accumulators: Option[String], // HTML
- val input: Option[TaskTableRowInputData],
- val output: Option[TaskTableRowOutputData],
- val shuffleRead: Option[TaskTableRowShuffleReadData],
- val shuffleWrite: Option[TaskTableRowShuffleWriteData],
- val bytesSpilled: Option[TaskTableRowBytesSpilledData],
- val error: String,
- val logs: Map[String, String])
-
private[ui] class TaskDataSource(
- tasks: Seq[TaskData],
- hasAccumulators: Boolean,
- hasInput: Boolean,
- hasOutput: Boolean,
- hasShuffleRead: Boolean,
- hasShuffleWrite: Boolean,
- hasBytesSpilled: Boolean,
+ stage: StageData,
currentTime: Long,
pageSize: Int,
sortColumn: String,
desc: Boolean,
- store: AppStatusStore) extends PagedDataSource[TaskTableRowData](pageSize) {
- import StagePage._
+ store: AppStatusStore) extends PagedDataSource[TaskData](pageSize) {
+ import ApiHelper._
// Keep an internal cache of executor log maps so that long task lists render faster.
private val executorIdToLogs = new HashMap[String, Map[String, String]]()
- // Convert TaskData to TaskTableRowData which contains the final contents to show in the table
- // so that we can avoid creating duplicate contents during sorting the data
- private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc))
-
- private var _slicedTaskIds: Set[Long] = _
-
- override def dataSize: Int = data.size
-
- override def sliceData(from: Int, to: Int): Seq[TaskTableRowData] = {
- val r = data.slice(from, to)
- _slicedTaskIds = r.map(_.taskId).toSet
- r
- }
-
- def slicedTaskIds: Set[Long] = _slicedTaskIds
+ private var _tasksToShow: Seq[TaskData] = null
- private def taskRow(info: TaskData): TaskTableRowData = {
- val metrics = info.taskMetrics
- val duration = info.duration.getOrElse(1L)
- val formatDuration = info.duration.map(d => UIUtils.formatDuration(d)).getOrElse("")
- val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L)
- val gcTime = metrics.map(_.jvmGcTime).getOrElse(0L)
- val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L)
- val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L)
- val gettingResultTime = getGettingResultTime(info, currentTime)
+ override def dataSize: Int = taskCount(stage)
- val externalAccumulableReadable = info.accumulatorUpdates.map { acc =>
- StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update}")
+ override def sliceData(from: Int, to: Int): Seq[TaskData] = {
+ if (_tasksToShow == null) {
+ _tasksToShow = store.taskList(stage.stageId, stage.attemptId, from, to - from,
+ indexName(sortColumn), !desc)
}
- val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L)
-
- val maybeInput = metrics.map(_.inputMetrics)
- val inputSortable = maybeInput.map(_.bytesRead).getOrElse(0L)
- val inputReadable = maybeInput
- .map(m => s"${Utils.bytesToString(m.bytesRead)}")
- .getOrElse("")
- val inputRecords = maybeInput.map(_.recordsRead.toString).getOrElse("")
-
- val maybeOutput = metrics.map(_.outputMetrics)
- val outputSortable = maybeOutput.map(_.bytesWritten).getOrElse(0L)
- val outputReadable = maybeOutput
- .map(m => s"${Utils.bytesToString(m.bytesWritten)}")
- .getOrElse("")
- val outputRecords = maybeOutput.map(_.recordsWritten.toString).getOrElse("")
-
- val maybeShuffleRead = metrics.map(_.shuffleReadMetrics)
- val shuffleReadBlockedTimeSortable = maybeShuffleRead.map(_.fetchWaitTime).getOrElse(0L)
- val shuffleReadBlockedTimeReadable =
- maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("")
-
- val totalShuffleBytes = maybeShuffleRead.map(ApiHelper.totalBytesRead)
- val shuffleReadSortable = totalShuffleBytes.getOrElse(0L)
- val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("")
- val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("")
-
- val remoteShuffleBytes = maybeShuffleRead.map(_.remoteBytesRead)
- val shuffleReadRemoteSortable = remoteShuffleBytes.getOrElse(0L)
- val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("")
-
- val maybeShuffleWrite = metrics.map(_.shuffleWriteMetrics)
- val shuffleWriteSortable = maybeShuffleWrite.map(_.bytesWritten).getOrElse(0L)
- val shuffleWriteReadable = maybeShuffleWrite
- .map(m => s"${Utils.bytesToString(m.bytesWritten)}").getOrElse("")
- val shuffleWriteRecords = maybeShuffleWrite
- .map(_.recordsWritten.toString).getOrElse("")
-
- val maybeWriteTime = metrics.map(_.shuffleWriteMetrics.writeTime)
- val writeTimeSortable = maybeWriteTime.getOrElse(0L)
- val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms =>
- if (ms == 0) "" else UIUtils.formatDuration(ms)
- }.getOrElse("")
-
- val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled)
- val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.getOrElse(0L)
- val memoryBytesSpilledReadable =
- maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("")
-
- val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled)
- val diskBytesSpilledSortable = maybeDiskBytesSpilled.getOrElse(0L)
- val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("")
-
- val input =
- if (hasInput) {
- Some(TaskTableRowInputData(inputSortable, s"$inputReadable / $inputRecords"))
- } else {
- None
- }
-
- val output =
- if (hasOutput) {
- Some(TaskTableRowOutputData(outputSortable, s"$outputReadable / $outputRecords"))
- } else {
- None
- }
-
- val shuffleRead =
- if (hasShuffleRead) {
- Some(TaskTableRowShuffleReadData(
- shuffleReadBlockedTimeSortable,
- shuffleReadBlockedTimeReadable,
- shuffleReadSortable,
- s"$shuffleReadReadable / $shuffleReadRecords",
- shuffleReadRemoteSortable,
- shuffleReadRemoteReadable
- ))
- } else {
- None
- }
-
- val shuffleWrite =
- if (hasShuffleWrite) {
- Some(TaskTableRowShuffleWriteData(
- writeTimeSortable,
- writeTimeReadable,
- shuffleWriteSortable,
- s"$shuffleWriteReadable / $shuffleWriteRecords"
- ))
- } else {
- None
- }
-
- val bytesSpilled =
- if (hasBytesSpilled) {
- Some(TaskTableRowBytesSpilledData(
- memoryBytesSpilledSortable,
- memoryBytesSpilledReadable,
- diskBytesSpilledSortable,
- diskBytesSpilledReadable
- ))
- } else {
- None
- }
-
- new TaskTableRowData(
- info.index,
- info.taskId,
- info.attempt,
- info.speculative,
- info.status,
- info.taskLocality.toString,
- info.executorId,
- info.host,
- info.launchTime.getTime(),
- duration,
- formatDuration,
- schedulerDelay,
- taskDeserializationTime,
- gcTime,
- serializationTime,
- gettingResultTime,
- peakExecutionMemoryUsed,
- if (hasAccumulators) Some(externalAccumulableReadable.mkString(" ")) else None,
- input,
- output,
- shuffleRead,
- shuffleWrite,
- bytesSpilled,
- info.errorMessage.getOrElse(""),
- executorLogs(info.executorId))
+ _tasksToShow
}
- private def executorLogs(id: String): Map[String, String] = {
+ def tasks: Seq[TaskData] = _tasksToShow
+
+ def executorLogs(id: String): Map[String, String] = {
executorIdToLogs.getOrElseUpdate(id,
store.asOption(store.executorSummary(id)).map(_.executorLogs).getOrElse(Map.empty))
}
- /**
- * Return Ordering according to sortColumn and desc
- */
- private def ordering(sortColumn: String, desc: Boolean): Ordering[TaskTableRowData] = {
- val ordering: Ordering[TaskTableRowData] = sortColumn match {
- case "Index" => Ordering.by(_.index)
- case "ID" => Ordering.by(_.taskId)
- case "Attempt" => Ordering.by(_.attempt)
- case "Status" => Ordering.by(_.status)
- case "Locality Level" => Ordering.by(_.taskLocality)
- case "Executor ID" => Ordering.by(_.executorId)
- case "Host" => Ordering.by(_.host)
- case "Launch Time" => Ordering.by(_.launchTime)
- case "Duration" => Ordering.by(_.duration)
- case "Scheduler Delay" => Ordering.by(_.schedulerDelay)
- case "Task Deserialization Time" => Ordering.by(_.taskDeserializationTime)
- case "GC Time" => Ordering.by(_.gcTime)
- case "Result Serialization Time" => Ordering.by(_.serializationTime)
- case "Getting Result Time" => Ordering.by(_.gettingResultTime)
- case "Peak Execution Memory" => Ordering.by(_.peakExecutionMemoryUsed)
- case "Accumulators" =>
- if (hasAccumulators) {
- Ordering.by(_.accumulators.get)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Accumulators because of no accumulators")
- }
- case "Input Size / Records" =>
- if (hasInput) {
- Ordering.by(_.input.get.inputSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Input Size / Records because of no inputs")
- }
- case "Output Size / Records" =>
- if (hasOutput) {
- Ordering.by(_.output.get.outputSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Output Size / Records because of no outputs")
- }
- // ShuffleRead
- case "Shuffle Read Blocked Time" =>
- if (hasShuffleRead) {
- Ordering.by(_.shuffleRead.get.shuffleReadBlockedTimeSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Read Blocked Time because of no shuffle reads")
- }
- case "Shuffle Read Size / Records" =>
- if (hasShuffleRead) {
- Ordering.by(_.shuffleRead.get.shuffleReadSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Read Size / Records because of no shuffle reads")
- }
- case "Shuffle Remote Reads" =>
- if (hasShuffleRead) {
- Ordering.by(_.shuffleRead.get.shuffleReadRemoteSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Remote Reads because of no shuffle reads")
- }
- // ShuffleWrite
- case "Write Time" =>
- if (hasShuffleWrite) {
- Ordering.by(_.shuffleWrite.get.writeTimeSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Write Time because of no shuffle writes")
- }
- case "Shuffle Write Size / Records" =>
- if (hasShuffleWrite) {
- Ordering.by(_.shuffleWrite.get.shuffleWriteSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Write Size / Records because of no shuffle writes")
- }
- // BytesSpilled
- case "Shuffle Spill (Memory)" =>
- if (hasBytesSpilled) {
- Ordering.by(_.bytesSpilled.get.memoryBytesSpilledSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Spill (Memory) because of no spills")
- }
- case "Shuffle Spill (Disk)" =>
- if (hasBytesSpilled) {
- Ordering.by(_.bytesSpilled.get.diskBytesSpilledSortable)
- } else {
- throw new IllegalArgumentException(
- "Cannot sort by Shuffle Spill (Disk) because of no spills")
- }
- case "Errors" => Ordering.by(_.error)
- case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn")
- }
- if (desc) {
- ordering.reverse
- } else {
- ordering
- }
- }
-
}
private[ui] class TaskPagedTable(
- conf: SparkConf,
+ stage: StageData,
basePath: String,
- data: Seq[TaskData],
- hasAccumulators: Boolean,
- hasInput: Boolean,
- hasOutput: Boolean,
- hasShuffleRead: Boolean,
- hasShuffleWrite: Boolean,
- hasBytesSpilled: Boolean,
currentTime: Long,
pageSize: Int,
sortColumn: String,
desc: Boolean,
- store: AppStatusStore) extends PagedTable[TaskTableRowData] {
+ store: AppStatusStore) extends PagedTable[TaskData] {
+
+ import ApiHelper._
override def tableId: String = "task-table"
@@ -1142,13 +719,7 @@ private[ui] class TaskPagedTable(
override def pageNumberFormField: String = "task.page"
override val dataSource: TaskDataSource = new TaskDataSource(
- data,
- hasAccumulators,
- hasInput,
- hasOutput,
- hasShuffleRead,
- hasShuffleWrite,
- hasBytesSpilled,
+ stage,
currentTime,
pageSize,
sortColumn,
@@ -1170,37 +741,39 @@ private[ui] class TaskPagedTable(
}
def headers: Seq[Node] = {
+ import ApiHelper._
+
val taskHeadersAndCssClasses: Seq[(String, String)] =
Seq(
- ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""),
- ("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", ""),
- ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY),
- ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME),
- ("GC Time", ""),
- ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME),
- ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME),
- ("Peak Execution Memory", TaskDetailsClassNames.PEAK_EXECUTION_MEMORY)) ++
- {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++
- {if (hasInput) Seq(("Input Size / Records", "")) else Nil} ++
- {if (hasOutput) Seq(("Output Size / Records", "")) else Nil} ++
- {if (hasShuffleRead) {
- Seq(("Shuffle Read Blocked Time", TaskDetailsClassNames.SHUFFLE_READ_BLOCKED_TIME),
- ("Shuffle Read Size / Records", ""),
- ("Shuffle Remote Reads", TaskDetailsClassNames.SHUFFLE_READ_REMOTE_SIZE))
+ (HEADER_TASK_INDEX, ""), (HEADER_ID, ""), (HEADER_ATTEMPT, ""), (HEADER_STATUS, ""),
+ (HEADER_LOCALITY, ""), (HEADER_EXECUTOR, ""), (HEADER_HOST, ""), (HEADER_LAUNCH_TIME, ""),
+ (HEADER_DURATION, ""), (HEADER_SCHEDULER_DELAY, TaskDetailsClassNames.SCHEDULER_DELAY),
+ (HEADER_DESER_TIME, TaskDetailsClassNames.TASK_DESERIALIZATION_TIME),
+ (HEADER_GC_TIME, ""),
+ (HEADER_SER_TIME, TaskDetailsClassNames.RESULT_SERIALIZATION_TIME),
+ (HEADER_GETTING_RESULT_TIME, TaskDetailsClassNames.GETTING_RESULT_TIME),
+ (HEADER_PEAK_MEM, TaskDetailsClassNames.PEAK_EXECUTION_MEMORY)) ++
+ {if (hasAccumulators(stage)) Seq((HEADER_ACCUMULATORS, "")) else Nil} ++
+ {if (hasInput(stage)) Seq((HEADER_INPUT_SIZE, "")) else Nil} ++
+ {if (hasOutput(stage)) Seq((HEADER_OUTPUT_SIZE, "")) else Nil} ++
+ {if (hasShuffleRead(stage)) {
+ Seq((HEADER_SHUFFLE_READ_TIME, TaskDetailsClassNames.SHUFFLE_READ_BLOCKED_TIME),
+ (HEADER_SHUFFLE_TOTAL_READS, ""),
+ (HEADER_SHUFFLE_REMOTE_READS, TaskDetailsClassNames.SHUFFLE_READ_REMOTE_SIZE))
} else {
Nil
}} ++
- {if (hasShuffleWrite) {
- Seq(("Write Time", ""), ("Shuffle Write Size / Records", ""))
+ {if (hasShuffleWrite(stage)) {
+ Seq((HEADER_SHUFFLE_WRITE_TIME, ""), (HEADER_SHUFFLE_WRITE_SIZE, ""))
} else {
Nil
}} ++
- {if (hasBytesSpilled) {
- Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", ""))
+ {if (hasBytesSpilled(stage)) {
+ Seq((HEADER_MEM_SPILL, ""), (HEADER_DISK_SPILL, ""))
} else {
Nil
}} ++
- Seq(("Errors", ""))
+ Seq((HEADER_ERROR, ""))
if (!taskHeadersAndCssClasses.map(_._1).contains(sortColumn)) {
throw new IllegalArgumentException(s"Unknown column: $sortColumn")
@@ -1237,7 +810,17 @@ private[ui] class TaskPagedTable(
{headerRow}
}
- def row(task: TaskTableRowData): Seq[Node] = {
+ def row(task: TaskData): Seq[Node] = {
+ def formatDuration(value: Option[Long], hideZero: Boolean = false): String = {
+ value.map { v =>
+ if (v > 0 || !hideZero) UIUtils.formatDuration(v) else ""
+ }.getOrElse("")
+ }
+
+ def formatBytes(value: Option[Long]): String = {
+ Utils.bytesToString(value.getOrElse(0L))
+ }
+
{task.index} |
{task.taskId} |
@@ -1249,62 +832,102 @@ private[ui] class TaskPagedTable(
{task.host}
{
- task.logs.map {
+ dataSource.executorLogs(task.executorId).map {
case (logName, logUrl) =>
}
}
- {UIUtils.formatDate(new Date(task.launchTime))} |
- {task.formatDuration} |
+ {UIUtils.formatDate(task.launchTime)} |
+ {formatDuration(task.duration)} |
- {UIUtils.formatDuration(task.schedulerDelay)}
+ {UIUtils.formatDuration(AppStatusUtils.schedulerDelay(task))}
|
- {UIUtils.formatDuration(task.taskDeserializationTime)}
+ {formatDuration(task.taskMetrics.map(_.executorDeserializeTime))}
|
- {if (task.gcTime > 0) UIUtils.formatDuration(task.gcTime) else ""}
+ {formatDuration(task.taskMetrics.map(_.jvmGcTime), hideZero = true)}
|
- {UIUtils.formatDuration(task.serializationTime)}
+ {formatDuration(task.taskMetrics.map(_.resultSerializationTime))}
|
- {UIUtils.formatDuration(task.gettingResultTime)}
+ {UIUtils.formatDuration(AppStatusUtils.gettingResultTime(task))}
|
- {Utils.bytesToString(task.peakExecutionMemoryUsed)}
+ {formatBytes(task.taskMetrics.map(_.peakExecutionMemory))}
|
- {if (task.accumulators.nonEmpty) {
- {Unparsed(task.accumulators.get)} |
+ {if (hasAccumulators(stage)) {
+ {accumulatorsInfo(task)} |
}}
- {if (task.input.nonEmpty) {
- {task.input.get.inputReadable} |
+ {if (hasInput(stage)) {
+ metricInfo(task) { m =>
+ val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead)
+ val records = m.inputMetrics.recordsRead
+ {bytesRead} / {records} |
+ }
}}
- {if (task.output.nonEmpty) {
- {task.output.get.outputReadable} |
+ {if (hasOutput(stage)) {
+ metricInfo(task) { m =>
+ val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten)
+ val records = m.outputMetrics.recordsWritten
+ {bytesWritten} / {records} |
+ }
}}
- {if (task.shuffleRead.nonEmpty) {
+ {if (hasShuffleRead(stage)) {
- {task.shuffleRead.get.shuffleReadBlockedTimeReadable}
+ {formatDuration(task.taskMetrics.map(_.shuffleReadMetrics.fetchWaitTime))}
|
- {task.shuffleRead.get.shuffleReadReadable} |
+ {
+ metricInfo(task) { m =>
+ val bytesRead = Utils.bytesToString(totalBytesRead(m.shuffleReadMetrics))
+ val records = m.shuffleReadMetrics.recordsRead
+ Unparsed(s"$bytesRead / $records")
+ }
+ } |
- {task.shuffleRead.get.shuffleReadRemoteReadable}
+ {formatBytes(task.taskMetrics.map(_.shuffleReadMetrics.remoteBytesRead))}
|
}}
- {if (task.shuffleWrite.nonEmpty) {
- {task.shuffleWrite.get.writeTimeReadable} |
- {task.shuffleWrite.get.shuffleWriteReadable} |
+ {if (hasShuffleWrite(stage)) {
+ {
+ formatDuration(
+ task.taskMetrics.map { m =>
+ TimeUnit.NANOSECONDS.toMillis(m.shuffleWriteMetrics.writeTime)
+ },
+ hideZero = true)
+ } |
+ {
+ metricInfo(task) { m =>
+ val bytesWritten = Utils.bytesToString(m.shuffleWriteMetrics.bytesWritten)
+ val records = m.shuffleWriteMetrics.recordsWritten
+ Unparsed(s"$bytesWritten / $records")
+ }
+ } |
}}
- {if (task.bytesSpilled.nonEmpty) {
- {task.bytesSpilled.get.memoryBytesSpilledReadable} |
- {task.bytesSpilled.get.diskBytesSpilledReadable} |
+ {if (hasBytesSpilled(stage)) {
+ {formatBytes(task.taskMetrics.map(_.memoryBytesSpilled))} |
+ {formatBytes(task.taskMetrics.map(_.diskBytesSpilled))} |
}}
- {errorMessageCell(task.error)}
+ {errorMessageCell(task.errorMessage.getOrElse(""))}
}
+ private def accumulatorsInfo(task: TaskData): Seq[Node] = {
+ task.accumulatorUpdates.flatMap { acc =>
+ if (acc.name != null && acc.update.isDefined) {
+ Unparsed(StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update.get}")) ++
+ } else {
+ Nil
+ }
+ }
+ }
+
+ private def metricInfo(task: TaskData)(fn: TaskMetrics => Seq[Node]): Seq[Node] = {
+ task.taskMetrics.map(fn).getOrElse(Nil)
+ }
+
private def errorMessageCell(error: String): Seq[Node] = {
val isMultiline = error.indexOf('\n') >= 0
// Display the first line by default
@@ -1331,7 +954,66 @@ private[ui] class TaskPagedTable(
}
}
-private object ApiHelper {
+private[ui] object ApiHelper {
+
+ val HEADER_ID = "ID"
+ val HEADER_TASK_INDEX = "Index"
+ val HEADER_ATTEMPT = "Attempt"
+ val HEADER_STATUS = "Status"
+ val HEADER_LOCALITY = "Locality Level"
+ val HEADER_EXECUTOR = "Executor ID"
+ val HEADER_HOST = "Host"
+ val HEADER_LAUNCH_TIME = "Launch Time"
+ val HEADER_DURATION = "Duration"
+ val HEADER_SCHEDULER_DELAY = "Scheduler Delay"
+ val HEADER_DESER_TIME = "Task Deserialization Time"
+ val HEADER_GC_TIME = "GC Time"
+ val HEADER_SER_TIME = "Result Serialization Time"
+ val HEADER_GETTING_RESULT_TIME = "Getting Result Time"
+ val HEADER_PEAK_MEM = "Peak Execution Memory"
+ val HEADER_ACCUMULATORS = "Accumulators"
+ val HEADER_INPUT_SIZE = "Input Size / Records"
+ val HEADER_OUTPUT_SIZE = "Output Size / Records"
+ val HEADER_SHUFFLE_READ_TIME = "Shuffle Read Blocked Time"
+ val HEADER_SHUFFLE_TOTAL_READS = "Shuffle Read Size / Records"
+ val HEADER_SHUFFLE_REMOTE_READS = "Shuffle Remote Reads"
+ val HEADER_SHUFFLE_WRITE_TIME = "Write Time"
+ val HEADER_SHUFFLE_WRITE_SIZE = "Shuffle Write Size / Records"
+ val HEADER_MEM_SPILL = "Shuffle Spill (Memory)"
+ val HEADER_DISK_SPILL = "Shuffle Spill (Disk)"
+ val HEADER_ERROR = "Errors"
+
+ private[ui] val COLUMN_TO_INDEX = Map(
+ HEADER_ID -> null.asInstanceOf[String],
+ HEADER_TASK_INDEX -> TaskIndexNames.TASK_INDEX,
+ HEADER_ATTEMPT -> TaskIndexNames.ATTEMPT,
+ HEADER_STATUS -> TaskIndexNames.STATUS,
+ HEADER_LOCALITY -> TaskIndexNames.LOCALITY,
+ HEADER_EXECUTOR -> TaskIndexNames.EXECUTOR,
+ HEADER_HOST -> TaskIndexNames.HOST,
+ HEADER_LAUNCH_TIME -> TaskIndexNames.LAUNCH_TIME,
+ HEADER_DURATION -> TaskIndexNames.DURATION,
+ HEADER_SCHEDULER_DELAY -> TaskIndexNames.SCHEDULER_DELAY,
+ HEADER_DESER_TIME -> TaskIndexNames.DESER_TIME,
+ HEADER_GC_TIME -> TaskIndexNames.GC_TIME,
+ HEADER_SER_TIME -> TaskIndexNames.SER_TIME,
+ HEADER_GETTING_RESULT_TIME -> TaskIndexNames.GETTING_RESULT_TIME,
+ HEADER_PEAK_MEM -> TaskIndexNames.PEAK_MEM,
+ HEADER_ACCUMULATORS -> TaskIndexNames.ACCUMULATORS,
+ HEADER_INPUT_SIZE -> TaskIndexNames.INPUT_SIZE,
+ HEADER_OUTPUT_SIZE -> TaskIndexNames.OUTPUT_SIZE,
+ HEADER_SHUFFLE_READ_TIME -> TaskIndexNames.SHUFFLE_READ_TIME,
+ HEADER_SHUFFLE_TOTAL_READS -> TaskIndexNames.SHUFFLE_TOTAL_READS,
+ HEADER_SHUFFLE_REMOTE_READS -> TaskIndexNames.SHUFFLE_REMOTE_READS,
+ HEADER_SHUFFLE_WRITE_TIME -> TaskIndexNames.SHUFFLE_WRITE_TIME,
+ HEADER_SHUFFLE_WRITE_SIZE -> TaskIndexNames.SHUFFLE_WRITE_SIZE,
+ HEADER_MEM_SPILL -> TaskIndexNames.MEM_SPILL,
+ HEADER_DISK_SPILL -> TaskIndexNames.DISK_SPILL,
+ HEADER_ERROR -> TaskIndexNames.ERROR)
+
+ def hasAccumulators(stageData: StageData): Boolean = {
+ stageData.accumulatorUpdates.exists { acc => acc.name != null && acc.value != null }
+ }
def hasInput(stageData: StageData): Boolean = stageData.inputBytes > 0
@@ -1349,4 +1031,21 @@ private object ApiHelper {
metrics.localBytesRead + metrics.remoteBytesRead
}
+ def indexName(sortColumn: String): Option[String] = {
+ COLUMN_TO_INDEX.get(sortColumn) match {
+ case Some(v) => Option(v)
+ case _ => throw new IllegalArgumentException(s"Invalid sort column: $sortColumn")
+ }
+ }
+
+ def lastStageNameAndDescription(store: AppStatusStore, job: JobData): (String, String) = {
+ val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0))
+ (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name))
+ }
+
+ def taskCount(stageData: StageData): Int = {
+ stageData.numActiveTasks + stageData.numCompleteTasks + stageData.numFailedTasks +
+ stageData.numKilledTasks
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index 18a4926f2f6c0..f001a01de3952 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -43,7 +43,9 @@ private[ui] class StageTableBase(
killEnabled: Boolean,
isFailedStage: Boolean) {
// stripXSS is called to remove suspicious characters used in XSS attacks
- val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS))
+ val allParameters = request.getParameterMap.asScala.toMap.map { case (k, v) =>
+ UIUtils.stripXSS(k) -> v.map(UIUtils.stripXSS).toSeq
+ }
val parameterOtherTable = allParameters.filterNot(_._1.startsWith(stageTag))
.map(para => para._1 + "=" + para._2(0))
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
index be05a963f0e68..10b032084ce4f 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
@@ -37,10 +37,10 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore)
attachPage(new PoolPage(this))
def isFairScheduler: Boolean = {
- store.environmentInfo().sparkProperties.toMap
- .get("spark.scheduler.mode")
- .map { mode => mode == SchedulingMode.FAIR }
- .getOrElse(false)
+ store
+ .environmentInfo()
+ .sparkProperties
+ .contains(("spark.scheduler.mode", SchedulingMode.FAIR.toString))
}
def handleKillRequest(request: HttpServletRequest): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
index 827a8637b9bd2..948858224d724 100644
--- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
+++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
@@ -116,7 +116,7 @@ private[spark] object RDDOperationGraph extends Logging {
// Use a special prefix here to differentiate this cluster from other operation clusters
val stageClusterId = STAGE_CLUSTER_PREFIX + stage.stageId
val stageClusterName = s"Stage ${stage.stageId}" +
- { if (stage.attemptId == 0) "" else s" (attempt ${stage.attemptId})" }
+ { if (stage.attemptNumber == 0) "" else s" (attempt ${stage.attemptNumber})" }
val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName)
var rootNodeCount = 0
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 02cee7f8c5b33..2674b9291203a 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -23,7 +23,7 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.{Node, Unparsed}
import org.apache.spark.status.AppStatusStore
-import org.apache.spark.status.api.v1.{RDDDataDistribution, RDDPartitionInfo}
+import org.apache.spark.status.api.v1.{ExecutorSummary, RDDDataDistribution, RDDPartitionInfo}
import org.apache.spark.ui._
import org.apache.spark.util.Utils
@@ -76,7 +76,8 @@ private[ui] class RDDPage(parent: SparkUITab, store: AppStatusStore) extends Web
rddStorageInfo.partitions.get,
blockPageSize,
blockSortColumn,
- blockSortDesc)
+ blockSortDesc,
+ store.executorList(true))
_blockTable.table(page)
} catch {
case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) =>
@@ -182,7 +183,8 @@ private[ui] class BlockDataSource(
rddPartitions: Seq[RDDPartitionInfo],
pageSize: Int,
sortColumn: String,
- desc: Boolean) extends PagedDataSource[BlockTableRowData](pageSize) {
+ desc: Boolean,
+ executorIdToAddress: Map[String, String]) extends PagedDataSource[BlockTableRowData](pageSize) {
private val data = rddPartitions.map(blockRow).sorted(ordering(sortColumn, desc))
@@ -198,7 +200,10 @@ private[ui] class BlockDataSource(
rddPartition.storageLevel,
rddPartition.memoryUsed,
rddPartition.diskUsed,
- rddPartition.executors.mkString(" "))
+ rddPartition.executors
+ .map { id => executorIdToAddress.get(id).getOrElse(id) }
+ .sorted
+ .mkString(" "))
}
/**
@@ -226,7 +231,8 @@ private[ui] class BlockPagedTable(
rddPartitions: Seq[RDDPartitionInfo],
pageSize: Int,
sortColumn: String,
- desc: Boolean) extends PagedTable[BlockTableRowData] {
+ desc: Boolean,
+ executorSummaries: Seq[ExecutorSummary]) extends PagedTable[BlockTableRowData] {
override def tableId: String = "rdd-storage-by-block-table"
@@ -243,7 +249,8 @@ private[ui] class BlockPagedTable(
rddPartitions,
pageSize,
sortColumn,
- desc)
+ desc,
+ executorSummaries.map { ex => (ex.id, ex.hostPort) }.toMap)
override def pageLink(page: Int): String = {
val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8")
diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala
index f4a736d6d439a..bf618b4afbce0 100644
--- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala
+++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala
@@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicLong
import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext}
+import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.AccumulableInfo
private[spark] case class AccumulatorMetadata(
@@ -199,10 +200,12 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable {
}
override def toString: String = {
+ // getClass.getSimpleName can cause Malformed class name error,
+ // call safer `Utils.getSimpleName` instead
if (metadata == null) {
- "Un-registered Accumulator: " + getClass.getSimpleName
+ "Un-registered Accumulator: " + Utils.getSimpleName(getClass)
} else {
- getClass.getSimpleName + s"(id: $id, name: $name, value: $value)"
+ Utils.getSimpleName(getClass) + s"(id: $id, name: $name, value: $value)"
}
}
}
@@ -211,7 +214,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable {
/**
* An internal class used to track accumulators by Spark itself.
*/
-private[spark] object AccumulatorContext {
+private[spark] object AccumulatorContext extends Logging {
/**
* This global map holds the original accumulator objects that are created on the driver.
@@ -258,13 +261,16 @@ private[spark] object AccumulatorContext {
* Returns the [[AccumulatorV2]] registered with the given ID, if any.
*/
def get(id: Long): Option[AccumulatorV2[_, _]] = {
- Option(originals.get(id)).map { ref =>
- // Since we are storing weak references, we must check whether the underlying data is valid.
+ val ref = originals.get(id)
+ if (ref eq null) {
+ None
+ } else {
+ // Since we are storing weak references, warn when the underlying data is not valid.
val acc = ref.get
if (acc eq null) {
- throw new IllegalStateException(s"Attempted to access garbage collected accumulator $id")
+ logWarning(s"Attempted to access garbage collected accumulator $id")
}
- acc
+ Option(acc)
}
}
@@ -290,7 +296,8 @@ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] {
private var _count = 0L
/**
- * Adds v to the accumulator, i.e. increment sum by v and count by 1.
+ * Returns false if this accumulator has had any values added to it or the sum is non-zero.
+ *
* @since 2.0.0
*/
override def isZero: Boolean = _sum == 0L && _count == 0
@@ -368,6 +375,9 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] {
private var _sum = 0.0
private var _count = 0L
+ /**
+ * Returns false if this accumulator has had any values added to it or the sum is non-zero.
+ */
override def isZero: Boolean = _sum == 0.0 && _count == 0
override def copy(): DoubleAccumulator = {
@@ -441,6 +451,9 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] {
class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] {
private val _list: java.util.List[T] = Collections.synchronizedList(new ArrayList[T]())
+ /**
+ * Returns false if this accumulator instance has any values in it.
+ */
override def isZero: Boolean = _list.isEmpty
override def copyAndReset(): CollectionAccumulator[T] = new CollectionAccumulator
@@ -479,7 +492,9 @@ class LegacyAccumulatorWrapper[R, T](
param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] {
private[spark] var _value = initialValue // Current value on driver
- override def isZero: Boolean = _value == param.zero(initialValue)
+ @transient private lazy val _zero = param.zero(initialValue)
+
+ override def isZero: Boolean = _value.asInstanceOf[AnyRef].eq(_zero.asInstanceOf[AnyRef])
override def copy(): LegacyAccumulatorWrapper[R, T] = {
val acc = new LegacyAccumulatorWrapper(initialValue, param)
@@ -488,7 +503,7 @@ class LegacyAccumulatorWrapper[R, T](
}
override def reset(): Unit = {
- _value = param.zero(initialValue)
+ _value = _zero
}
override def add(v: T): Unit = _value = param.addAccumulator(_value, v)
diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
index 31d230d0fec8e..21acaa95c5645 100644
--- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
+++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
@@ -22,9 +22,7 @@ package org.apache.spark.util
* through all the elements.
*/
private[spark]
-// scalastyle:off
abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] {
-// scalastyle:on
private[this] var completed = false
def next(): A = sub.next()
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 5e60218c5740b..ff83301d631c4 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -263,7 +263,7 @@ private[spark] object JsonProtocol {
val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing)
val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing)
("Stage ID" -> stageInfo.stageId) ~
- ("Stage Attempt ID" -> stageInfo.attemptId) ~
+ ("Stage Attempt ID" -> stageInfo.attemptNumber) ~
("Stage Name" -> stageInfo.name) ~
("Number of Tasks" -> stageInfo.numTasks) ~
("RDD Info" -> rddInfo) ~
diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
index 76a56298aaebc..4a7798434680e 100644
--- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
@@ -60,6 +60,15 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging {
}
}
+ /**
+ * This can be overriden by subclasses if there is any extra cleanup to do when removing a
+ * listener. In particular AsyncEventQueues can clean up queues in the LiveListenerBus.
+ */
+ def removeListenerOnError(listener: L): Unit = {
+ removeListener(listener)
+ }
+
+
/**
* Post the event to all registered listeners. The `postToAll` caller should guarantee calling
* `postToAll` in the same thread for all events.
@@ -80,7 +89,16 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging {
}
try {
doPostEvent(listener, event)
+ if (Thread.interrupted()) {
+ // We want to throw the InterruptedException right away so we can associate the interrupt
+ // with this listener, as opposed to waiting for a queue.take() etc. to detect it.
+ throw new InterruptedException()
+ }
} catch {
+ case ie: InterruptedException =>
+ logError(s"Interrupted while posting to ${Utils.getFormattedClassName(listener)}. " +
+ s"Removing that listener.", ie)
+ removeListenerOnError(listener)
case NonFatal(e) =>
logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e)
} finally {
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 5853302973140..d4b72e8474626 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -18,6 +18,8 @@
package org.apache.spark.util
import java.io._
+import java.lang.{Byte => JByte}
+import java.lang.InternalError
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
@@ -26,6 +28,7 @@ import java.nio.ByteBuffer
import java.nio.channels.{Channels, FileChannel}
import java.nio.charset.StandardCharsets
import java.nio.file.{Files, Paths}
+import java.security.SecureRandom
import java.util.{Locale, Properties, Random, UUID}
import java.util.concurrent._
import java.util.concurrent.atomic.AtomicBoolean
@@ -44,6 +47,7 @@ import scala.util.matching.Regex
import _root_.io.netty.channel.unix.Errors.NativeIoException
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
+import com.google.common.hash.HashCodes
import com.google.common.io.{ByteStreams, Files => GFiles}
import com.google.common.net.InetAddresses
import org.apache.commons.lang3.SystemUtils
@@ -1872,7 +1876,7 @@ private[spark] object Utils extends Logging {
/** Return the class name of the given object, removing all dollar signs */
def getFormattedClassName(obj: AnyRef): String = {
- obj.getClass.getSimpleName.replace("$", "")
+ getSimpleName(obj.getClass).replace("$", "")
}
/** Return an option that translates JNothing to None */
@@ -2805,6 +2809,71 @@ private[spark] object Utils extends Logging {
s"k8s://$resolvedURL"
}
+
+ def createSecret(conf: SparkConf): String = {
+ val bits = conf.get(AUTH_SECRET_BIT_LENGTH)
+ val rnd = new SecureRandom()
+ val secretBytes = new Array[Byte](bits / JByte.SIZE)
+ rnd.nextBytes(secretBytes)
+ HashCodes.fromBytes(secretBytes).toString()
+ }
+
+ /**
+ * Safer than Class obj's getSimpleName which may throw Malformed class name error in scala.
+ * This method mimicks scalatest's getSimpleNameOfAnObjectsClass.
+ */
+ def getSimpleName(cls: Class[_]): String = {
+ try {
+ return cls.getSimpleName
+ } catch {
+ case err: InternalError => return stripDollars(stripPackages(cls.getName))
+ }
+ }
+
+ /**
+ * Remove the packages from full qualified class name
+ */
+ private def stripPackages(fullyQualifiedName: String): String = {
+ fullyQualifiedName.split("\\.").takeRight(1)(0)
+ }
+
+ /**
+ * Remove trailing dollar signs from qualified class name,
+ * and return the trailing part after the last dollar sign in the middle
+ */
+ private def stripDollars(s: String): String = {
+ val lastDollarIndex = s.lastIndexOf('$')
+ if (lastDollarIndex < s.length - 1) {
+ // The last char is not a dollar sign
+ if (lastDollarIndex == -1 || !s.contains("$iw")) {
+ // The name does not have dollar sign or is not an intepreter
+ // generated class, so we should return the full string
+ s
+ } else {
+ // The class name is intepreter generated,
+ // return the part after the last dollar sign
+ // This is the same behavior as getClass.getSimpleName
+ s.substring(lastDollarIndex + 1)
+ }
+ }
+ else {
+ // The last char is a dollar sign
+ // Find last non-dollar char
+ val lastNonDollarChar = s.reverse.find(_ != '$')
+ lastNonDollarChar match {
+ case None => s
+ case Some(c) =>
+ val lastNonDollarIndex = s.lastIndexOf(c)
+ if (lastNonDollarIndex == -1) {
+ s
+ } else {
+ // Strip the trailing dollar signs
+ // Invoke stripDollars again to get the simple name
+ stripDollars(s.substring(0, lastNonDollarIndex + 1))
+ }
+ }
+ }
+ }
}
private[util] object CallerContext extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 375f4a6921225..5c6dd45ec58e3 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -463,7 +463,7 @@ class ExternalAppendOnlyMap[K, V, C](
// An intermediate stream that reads from exactly one batch
// This guards against pre-fetching and other arbitrary behavior of higher level streams
- private var deserializeStream = nextBatchStream()
+ private var deserializeStream: DeserializationStream = null
private var nextItem: (K, C) = null
private var objectsRead = 0
@@ -528,7 +528,11 @@ class ExternalAppendOnlyMap[K, V, C](
override def hasNext: Boolean = {
if (nextItem == null) {
if (deserializeStream == null) {
- return false
+ // In case of deserializeStream has not been initialized
+ deserializeStream = nextBatchStream()
+ if (deserializeStream == null) {
+ return false
+ }
}
nextItem = readNextItem()
}
@@ -536,19 +540,18 @@ class ExternalAppendOnlyMap[K, V, C](
}
override def next(): (K, C) = {
- val item = if (nextItem == null) readNextItem() else nextItem
- if (item == null) {
+ if (!hasNext) {
throw new NoSuchElementException
}
+ val item = nextItem
nextItem = null
item
}
private def cleanup() {
batchIndex = batchOffsets.length // Prevent reading any other batch
- val ds = deserializeStream
- if (ds != null) {
- ds.close()
+ if (deserializeStream != null) {
+ deserializeStream.close()
deserializeStream = null
}
if (fileStream != null) {
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 7367af7888bd8..3ae8dfcc1cb66 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
@@ -63,10 +63,15 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
*/
def writeFully(channel: WritableByteChannel): Unit = {
for (bytes <- getChunks()) {
- while (bytes.remaining() > 0) {
- val ioSize = Math.min(bytes.remaining(), bufferWriteChunkSize)
- bytes.limit(bytes.position() + ioSize)
- channel.write(bytes)
+ val curChunkLimit = bytes.limit()
+ while (bytes.hasRemaining) {
+ try {
+ val ioSize = Math.min(bytes.remaining(), bufferWriteChunkSize)
+ bytes.limit(bytes.position() + ioSize)
+ channel.write(bytes)
+ } finally {
+ bytes.limit(curChunkLimit)
+ }
}
}
}
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 c2261c204cd45..2225591a4ff75 100644
--- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
+++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -17,6 +17,7 @@
package org.apache.spark.launcher;
+import java.time.Duration;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.HashMap;
@@ -30,6 +31,7 @@
import static org.mockito.Mockito.*;
import org.apache.spark.SparkContext;
+import org.apache.spark.SparkContext$;
import org.apache.spark.internal.config.package$;
import org.apache.spark.util.Utils;
@@ -133,6 +135,12 @@ public void testInProcessLauncher() throws Exception {
p.put(e.getKey(), e.getValue());
}
System.setProperties(p);
+ // Here DAGScheduler is stopped, while SparkContext.clearActiveContext may not be called yet.
+ // Wait for a reasonable amount of time to avoid creating two active SparkContext in JVM.
+ // See SPARK-23019 and SparkContext.stop() for details.
+ eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> {
+ assertTrue("SparkContext is still alive.", SparkContext$.MODULE$.getActive().isEmpty());
+ });
}
}
@@ -141,26 +149,47 @@ private void inProcessLauncherTestImpl() throws Exception {
SparkAppHandle.Listener listener = mock(SparkAppHandle.Listener.class);
doAnswer(invocation -> {
SparkAppHandle h = (SparkAppHandle) invocation.getArguments()[0];
- transitions.add(h.getState());
+ synchronized (transitions) {
+ transitions.add(h.getState());
+ }
return null;
}).when(listener).stateChanged(any(SparkAppHandle.class));
- SparkAppHandle handle = new InProcessLauncher()
- .setMaster("local")
- .setAppResource(SparkLauncher.NO_RESOURCE)
- .setMainClass(InProcessTestApp.class.getName())
- .addAppArgs("hello")
- .startApplication(listener);
-
- waitFor(handle);
- assertEquals(SparkAppHandle.State.FINISHED, handle.getState());
-
- // Matches the behavior of LocalSchedulerBackend.
- List expected = Arrays.asList(
- SparkAppHandle.State.CONNECTED,
- SparkAppHandle.State.RUNNING,
- SparkAppHandle.State.FINISHED);
- assertEquals(expected, transitions);
+ SparkAppHandle handle = null;
+ try {
+ synchronized (InProcessTestApp.LOCK) {
+ handle = new InProcessLauncher()
+ .setMaster("local")
+ .setAppResource(SparkLauncher.NO_RESOURCE)
+ .setMainClass(InProcessTestApp.class.getName())
+ .addAppArgs("hello")
+ .startApplication(listener);
+
+ // SPARK-23020: see doc for InProcessTestApp.LOCK for a description of the race. Here
+ // we wait until we know that the connection between the app and the launcher has been
+ // established before allowing the app to finish.
+ final SparkAppHandle _handle = handle;
+ eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> {
+ assertNotEquals(SparkAppHandle.State.UNKNOWN, _handle.getState());
+ });
+
+ InProcessTestApp.LOCK.wait(5000);
+ }
+
+ waitFor(handle);
+ assertEquals(SparkAppHandle.State.FINISHED, handle.getState());
+
+ // Matches the behavior of LocalSchedulerBackend.
+ List expected = Arrays.asList(
+ SparkAppHandle.State.CONNECTED,
+ SparkAppHandle.State.RUNNING,
+ SparkAppHandle.State.FINISHED);
+ assertEquals(expected, transitions);
+ } finally {
+ if (handle != null) {
+ handle.kill();
+ }
+ }
}
public static class SparkLauncherTestApp {
@@ -176,10 +205,26 @@ public static void main(String[] args) throws Exception {
public static class InProcessTestApp {
+ /**
+ * SPARK-23020: there's a race caused by a child app finishing too quickly. This would cause
+ * the InProcessAppHandle to dispose of itself even before the child connection was properly
+ * established, so no state changes would be detected for the application and its final
+ * state would be LOST.
+ *
+ * It's not really possible to fix that race safely in the handle code itself without changing
+ * the way in-process apps talk to the launcher library, so we work around that in the test by
+ * synchronizing on this object.
+ */
+ public static final Object LOCK = new Object();
+
public static void main(String[] args) throws Exception {
assertNotEquals(0, args.length);
assertEquals(args[0], "hello");
new SparkContext().stop();
+
+ synchronized (LOCK) {
+ LOCK.notifyAll();
+ }
}
}
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index 46b0516e36141..a0664b30d6cc2 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -21,6 +21,7 @@
import org.junit.Test;
import org.apache.spark.SparkConf;
+import org.apache.spark.unsafe.memory.MemoryAllocator;
import org.apache.spark.unsafe.memory.MemoryBlock;
public class TaskMemoryManagerSuite {
@@ -68,6 +69,34 @@ public void encodePageNumberAndOffsetOnHeap() {
Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress));
}
+ @Test
+ public void freeingPageSetsPageNumberToSpecialConstant() {
+ final TaskMemoryManager manager = new TaskMemoryManager(
+ new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+ final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
+ final MemoryBlock dataPage = manager.allocatePage(256, c);
+ c.freePage(dataPage);
+ Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber);
+ }
+
+ @Test(expected = AssertionError.class)
+ public void freeingPageDirectlyInAllocatorTriggersAssertionError() {
+ final TaskMemoryManager manager = new TaskMemoryManager(
+ new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+ final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
+ final MemoryBlock dataPage = manager.allocatePage(256, c);
+ MemoryAllocator.HEAP.free(dataPage);
+ }
+
+ @Test(expected = AssertionError.class)
+ public void callingFreePageOnDirectlyAllocatedPageTriggersAssertionError() {
+ final TaskMemoryManager manager = new TaskMemoryManager(
+ new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+ final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
+ final MemoryBlock dataPage = MemoryAllocator.HEAP.allocate(256);
+ manager.freePage(dataPage, c);
+ }
+
@Test
public void cooperativeSpilling() {
final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf());
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
index af4975c888d65..411cd5cb57331 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
@@ -72,8 +72,10 @@ public class UnsafeExternalSorterSuite {
public int compare(
Object leftBaseObject,
long leftBaseOffset,
+ int leftBaseLength,
Object rightBaseObject,
- long rightBaseOffset) {
+ long rightBaseOffset,
+ int rightBaseLength) {
return 0;
}
};
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
index 594f07dd780f9..85ffdca436e14 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
@@ -98,8 +98,10 @@ public void testSortingOnlyByIntegerPrefix() throws Exception {
public int compare(
Object leftBaseObject,
long leftBaseOffset,
+ int leftBaseLength,
Object rightBaseObject,
- long rightBaseOffset) {
+ long rightBaseOffset,
+ int rightBaseLength) {
return 0;
}
};
@@ -127,7 +129,6 @@ public int compare(
final UnsafeSorterIterator iter = sorter.getSortedIterator();
int iterLength = 0;
long prevPrefix = -1;
- Arrays.sort(dataToSort);
while (iter.hasNext()) {
iter.loadNext();
final String str =
@@ -164,8 +165,10 @@ public void freeAfterOOM() {
public int compare(
Object leftBaseObject,
long leftBaseOffset,
+ int leftBaseLength,
Object rightBaseObject,
- long rightBaseOffset) {
+ long rightBaseOffset,
+ int rightBaseLength) {
return 0;
}
};
diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
index 94f5805853e1e..f8e233a05a447 100644
--- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
+++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
@@ -38,6 +38,7 @@ public static void test() {
tc.attemptNumber();
tc.partitionId();
tc.stageId();
+ tc.stageAttemptNumber();
tc.taskAttemptId();
}
@@ -51,6 +52,7 @@ public void onTaskCompletion(TaskContext context) {
context.isCompleted();
context.isInterrupted();
context.stageId();
+ context.stageAttemptNumber();
context.partitionId();
context.addTaskCompletionListener(this);
}
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json
index f8e27703c0def..5c42ac1d87f4c 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json
@@ -7,6 +7,9 @@
"resultSize" : [ 2010.0, 2065.0, 2065.0 ],
"jvmGcTime" : [ 0.0, 0.0, 7.0 ],
"resultSerializationTime" : [ 0.0, 0.0, 2.0 ],
+ "gettingResultTime" : [ 0.0, 0.0, 0.0 ],
+ "schedulerDelay" : [ 2.0, 6.0, 53.0 ],
+ "peakExecutionMemory" : [ 0.0, 0.0, 0.0 ],
"memoryBytesSpilled" : [ 0.0, 0.0, 0.0 ],
"diskBytesSpilled" : [ 0.0, 0.0, 0.0 ],
"inputMetrics" : {
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json
index a28bda16a956e..e6b705989cc97 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json
@@ -7,6 +7,9 @@
"resultSize" : [ 1034.0, 1034.0, 1034.0, 1034.0, 1034.0 ],
"jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+ "gettingResultTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+ "schedulerDelay" : [ 4.0, 4.0, 6.0, 7.0, 9.0 ],
+ "peakExecutionMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"inputMetrics" : {
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json
index ede3eaed1d1d2..788f28cf7b365 100644
--- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json
@@ -7,6 +7,9 @@
"resultSize" : [ 2010.0, 2065.0, 2065.0, 2065.0, 2065.0 ],
"jvmGcTime" : [ 0.0, 0.0, 0.0, 5.0, 7.0 ],
"resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 1.0 ],
+ "gettingResultTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+ "schedulerDelay" : [ 2.0, 4.0, 6.0, 13.0, 40.0 ],
+ "peakExecutionMemory" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
"inputMetrics" : {
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 3990ee1ec326d..5d0ffd92647bc 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -209,10 +209,8 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex
System.gc()
assert(ref.get.isEmpty)
- // Getting a garbage collected accum should throw error
- intercept[IllegalStateException] {
- AccumulatorContext.get(accId)
- }
+ // Getting a garbage collected accum should return None.
+ assert(AccumulatorContext.get(accId).isEmpty)
// Getting a normal accumulator. Note: this has to be separate because referencing an
// accumulator above in an `assert` would keep it from being garbage collected.
diff --git a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala
index 91355f7362900..a5bdc95790722 100644
--- a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala
+++ b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala
@@ -103,8 +103,11 @@ class DebugFilesystem extends LocalFileSystem {
override def markSupported(): Boolean = wrapped.markSupported()
override def close(): Unit = {
- wrapped.close()
- removeOpenStream(wrapped)
+ try {
+ wrapped.close()
+ } finally {
+ removeOpenStream(wrapped)
+ }
}
override def read(): Int = wrapped.read()
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index a0cae5a9e011c..9807d1269e3d4 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -19,6 +19,8 @@ package org.apache.spark
import scala.collection.mutable
+import org.mockito.Matchers.{any, eq => meq}
+import org.mockito.Mockito.{mock, never, verify, when}
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
import org.apache.spark.executor.TaskMetrics
@@ -26,6 +28,7 @@ import org.apache.spark.scheduler._
import org.apache.spark.scheduler.ExternalClusterManager
import org.apache.spark.scheduler.cluster.ExecutorInfo
import org.apache.spark.scheduler.local.LocalSchedulerBackend
+import org.apache.spark.storage.BlockManagerMaster
import org.apache.spark.util.ManualClock
/**
@@ -1050,6 +1053,66 @@ class ExecutorAllocationManagerSuite
assert(removeTimes(manager) === Map.empty)
}
+ test("SPARK-23365 Don't update target num executors when killing idle executors") {
+ val minExecutors = 1
+ val initialExecutors = 1
+ val maxExecutors = 2
+ val conf = new SparkConf()
+ .set("spark.dynamicAllocation.enabled", "true")
+ .set("spark.shuffle.service.enabled", "true")
+ .set("spark.dynamicAllocation.minExecutors", minExecutors.toString)
+ .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString)
+ .set("spark.dynamicAllocation.initialExecutors", initialExecutors.toString)
+ .set("spark.dynamicAllocation.schedulerBacklogTimeout", "1000ms")
+ .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", "1000ms")
+ .set("spark.dynamicAllocation.executorIdleTimeout", s"3000ms")
+ val mockAllocationClient = mock(classOf[ExecutorAllocationClient])
+ val mockBMM = mock(classOf[BlockManagerMaster])
+ val manager = new ExecutorAllocationManager(
+ mockAllocationClient, mock(classOf[LiveListenerBus]), conf, mockBMM)
+ val clock = new ManualClock()
+ manager.setClock(clock)
+
+ when(mockAllocationClient.requestTotalExecutors(meq(2), any(), any())).thenReturn(true)
+ // test setup -- job with 2 tasks, scale up to two executors
+ assert(numExecutorsTarget(manager) === 1)
+ manager.listener.onExecutorAdded(SparkListenerExecutorAdded(
+ clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
+ manager.listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 2)))
+ clock.advance(1000)
+ manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis())
+ assert(numExecutorsTarget(manager) === 2)
+ val taskInfo0 = createTaskInfo(0, 0, "executor-1")
+ manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo0))
+ manager.listener.onExecutorAdded(SparkListenerExecutorAdded(
+ clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty)))
+ val taskInfo1 = createTaskInfo(1, 1, "executor-2")
+ manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo1))
+ assert(numExecutorsTarget(manager) === 2)
+
+ // have one task finish -- we should adjust the target number of executors down
+ // but we should *not* kill any executors yet
+ manager.listener.onTaskEnd(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, null))
+ assert(maxNumExecutorsNeeded(manager) === 1)
+ assert(numExecutorsTarget(manager) === 2)
+ clock.advance(1000)
+ manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis())
+ assert(numExecutorsTarget(manager) === 1)
+ verify(mockAllocationClient, never).killExecutors(any(), any(), any(), any())
+
+ // now we cross the idle timeout for executor-1, so we kill it. the really important
+ // thing here is that we do *not* ask the executor allocation client to adjust the target
+ // number of executors down
+ when(mockAllocationClient.killExecutors(Seq("executor-1"), false, false, false))
+ .thenReturn(Seq("executor-1"))
+ clock.advance(3000)
+ schedule(manager)
+ assert(maxNumExecutorsNeeded(manager) === 1)
+ assert(numExecutorsTarget(manager) === 1)
+ // here's the important verify -- we did kill the executors, but did not adjust the target count
+ verify(mockAllocationClient).killExecutors(Seq("executor-1"), false, false, false)
+ }
+
private def createSparkContext(
minExecutors: Int = 1,
maxExecutors: Int = 5,
@@ -1268,7 +1331,8 @@ private class DummyLocalSchedulerBackend (sc: SparkContext, sb: SchedulerBackend
override def killExecutors(
executorIds: Seq[String],
- replace: Boolean,
+ adjustTargetNumExecutors: Boolean,
+ countFailures: Boolean,
force: Boolean): Seq[String] = executorIds
override def start(): Unit = sb.start()
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index e9539dc73f6fa..55a9122cf9026 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -244,7 +244,10 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
for (i <- 0 until testOutputCopies) {
// Shift values by i so that they're different in the output
val alteredOutput = testOutput.map(b => (b + i).toByte)
- channel.write(ByteBuffer.wrap(alteredOutput))
+ val buffer = ByteBuffer.wrap(alteredOutput)
+ while (buffer.hasRemaining) {
+ channel.write(buffer)
+ }
}
channel.close()
file.close()
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index 8a77aea75a992..61da4138896cd 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark
import java.util.concurrent.Semaphore
+import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.Future
@@ -26,7 +27,7 @@ import scala.concurrent.duration._
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers
-import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart}
import org.apache.spark.util.ThreadUtils
/**
@@ -40,6 +41,10 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
override def afterEach() {
try {
resetSparkContext()
+ JobCancellationSuite.taskStartedSemaphore.drainPermits()
+ JobCancellationSuite.taskCancelledSemaphore.drainPermits()
+ JobCancellationSuite.twoJobsSharingStageSemaphore.drainPermits()
+ JobCancellationSuite.executionOfInterruptibleCounter.set(0)
} finally {
super.afterEach()
}
@@ -320,6 +325,67 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
f2.get()
}
+ test("interruptible iterator of shuffle reader") {
+ // In this test case, we create a Spark job of two stages. The second stage is cancelled during
+ // execution and a counter is used to make sure that the corresponding tasks are indeed
+ // cancelled.
+ import JobCancellationSuite._
+ sc = new SparkContext("local[2]", "test interruptible iterator")
+
+ // Increase the number of elements to be proceeded to avoid this test being flaky.
+ val numElements = 10000
+ val taskCompletedSem = new Semaphore(0)
+
+ sc.addSparkListener(new SparkListener {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
+ // release taskCancelledSemaphore when cancelTasks event has been posted
+ if (stageCompleted.stageInfo.stageId == 1) {
+ taskCancelledSemaphore.release(numElements)
+ }
+ }
+
+ override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
+ if (taskEnd.stageId == 1) { // make sure tasks are completed
+ taskCompletedSem.release()
+ }
+ }
+ })
+
+ // Explicitly disable interrupt task thread on cancelling tasks, so the task thread can only be
+ // interrupted by `InterruptibleIterator`.
+ sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false")
+
+ val f = sc.parallelize(1 to numElements).map { i => (i, i) }
+ .repartitionAndSortWithinPartitions(new HashPartitioner(1))
+ .mapPartitions { iter =>
+ taskStartedSemaphore.release()
+ iter
+ }.foreachAsync { x =>
+ // Block this code from being executed, until the job get cancelled. In this case, if the
+ // source iterator is interruptible, the max number of increment should be under
+ // `numElements`.
+ taskCancelledSemaphore.acquire()
+ executionOfInterruptibleCounter.getAndIncrement()
+ }
+
+ taskStartedSemaphore.acquire()
+ // Job is cancelled when:
+ // 1. task in reduce stage has been started, guaranteed by previous line.
+ // 2. task in reduce stage is blocked as taskCancelledSemaphore is not released until
+ // JobCancelled event is posted.
+ // After job being cancelled, task in reduce stage will be cancelled asynchronously, thus
+ // partial of the inputs should not get processed (It's very unlikely that Spark can process
+ // 10000 elements between JobCancelled is posted and task is really killed).
+ f.cancel()
+
+ val e = intercept[SparkException](f.get()).getCause
+ assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+
+ // Make sure tasks are indeed completed.
+ taskCompletedSem.acquire()
+ assert(executionOfInterruptibleCounter.get() < numElements)
+ }
+
def testCount() {
// Cancel before launching any tasks
{
@@ -381,7 +447,9 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
object JobCancellationSuite {
+ // To avoid any headaches, reset these global variables in the companion class's afterEach block
val taskStartedSemaphore = new Semaphore(0)
val taskCancelledSemaphore = new Semaphore(0)
val twoJobsSharingStageSemaphore = new Semaphore(0)
+ val executionOfInterruptibleCounter = new AtomicInteger(0)
}
diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
index 155ca17db726b..9206b5debf4f3 100644
--- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
@@ -262,14 +262,11 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva
test("defaultPartitioner") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150)
- val rdd2 = sc
- .parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
+ val rdd2 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(10))
- val rdd3 = sc
- .parallelize(Array((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
+ val rdd3 = sc.parallelize(Array((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
.partitionBy(new HashPartitioner(100))
- val rdd4 = sc
- .parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
+ val rdd4 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(9))
val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11)
@@ -284,7 +281,42 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva
assert(partitioner3.numPartitions == rdd3.getNumPartitions)
assert(partitioner4.numPartitions == rdd3.getNumPartitions)
assert(partitioner5.numPartitions == rdd4.getNumPartitions)
+ }
+ test("defaultPartitioner when defaultParallelism is set") {
+ assert(!sc.conf.contains("spark.default.parallelism"))
+ try {
+ sc.conf.set("spark.default.parallelism", "4")
+
+ val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150)
+ val rdd2 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
+ .partitionBy(new HashPartitioner(10))
+ val rdd3 = sc.parallelize(Array((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
+ .partitionBy(new HashPartitioner(100))
+ val rdd4 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
+ .partitionBy(new HashPartitioner(9))
+ val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11)
+ val rdd6 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
+ .partitionBy(new HashPartitioner(3))
+
+ val partitioner1 = Partitioner.defaultPartitioner(rdd1, rdd2)
+ val partitioner2 = Partitioner.defaultPartitioner(rdd2, rdd3)
+ val partitioner3 = Partitioner.defaultPartitioner(rdd3, rdd1)
+ val partitioner4 = Partitioner.defaultPartitioner(rdd1, rdd2, rdd3)
+ val partitioner5 = Partitioner.defaultPartitioner(rdd4, rdd5)
+ val partitioner6 = Partitioner.defaultPartitioner(rdd5, rdd5)
+ val partitioner7 = Partitioner.defaultPartitioner(rdd1, rdd6)
+
+ assert(partitioner1.numPartitions == rdd2.getNumPartitions)
+ assert(partitioner2.numPartitions == rdd3.getNumPartitions)
+ assert(partitioner3.numPartitions == rdd3.getNumPartitions)
+ assert(partitioner4.numPartitions == rdd3.getNumPartitions)
+ assert(partitioner5.numPartitions == rdd4.getNumPartitions)
+ assert(partitioner6.numPartitions == sc.defaultParallelism)
+ assert(partitioner7.numPartitions == sc.defaultParallelism)
+ } finally {
+ sc.conf.remove("spark.default.parallelism")
+ }
}
}
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index 3931d53b4ae0a..ced5a06516f75 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -363,14 +363,14 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
// first attempt -- its successful
val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0,
- new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem))
+ new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem))
val data1 = (1 to 10).map { x => x -> x}
// second attempt -- also successful. We'll write out different data,
// just to simulate the fact that the records may get written differently
// depending on what gets spilled, what gets combined, etc.
val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0,
- new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem))
+ new TaskContextImpl(0, 0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem))
val data2 = (11 to 20).map { x => x -> x}
// interleave writes of both attempts -- we want to test that both attempts can occur
@@ -398,7 +398,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
}
val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1,
- new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem))
+ new TaskContextImpl(1, 0, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem))
val readData = reader.read().toIndexedSeq
assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq)
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index b30bd74812b36..ce9f2be1c02dd 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark
import java.io.File
import java.net.{MalformedURLException, URI}
import java.nio.charset.StandardCharsets
-import java.util.concurrent.{Semaphore, TimeUnit}
+import java.util.concurrent.{CountDownLatch, Semaphore, TimeUnit}
import scala.concurrent.duration._
@@ -498,45 +498,36 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
test("Cancelling stages/jobs with custom reasons.") {
sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+ sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true")
val REASON = "You shall not pass"
- val slices = 10
- val listener = new SparkListener {
- override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
- if (SparkContextSuite.cancelStage) {
- eventually(timeout(10.seconds)) {
- assert(SparkContextSuite.isTaskStarted)
+ for (cancelWhat <- Seq("stage", "job")) {
+ // This countdown latch used to make sure stage or job canceled in listener
+ val latch = new CountDownLatch(1)
+
+ val listener = cancelWhat match {
+ case "stage" =>
+ new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
+ sc.cancelStage(taskStart.stageId, REASON)
+ latch.countDown()
+ }
}
- sc.cancelStage(taskStart.stageId, REASON)
- SparkContextSuite.cancelStage = false
- SparkContextSuite.semaphore.release(slices)
- }
- }
-
- override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
- if (SparkContextSuite.cancelJob) {
- eventually(timeout(10.seconds)) {
- assert(SparkContextSuite.isTaskStarted)
+ case "job" =>
+ new SparkListener {
+ override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
+ sc.cancelJob(jobStart.jobId, REASON)
+ latch.countDown()
+ }
}
- sc.cancelJob(jobStart.jobId, REASON)
- SparkContextSuite.cancelJob = false
- SparkContextSuite.semaphore.release(slices)
- }
}
- }
- sc.addSparkListener(listener)
-
- for (cancelWhat <- Seq("stage", "job")) {
- SparkContextSuite.semaphore.drainPermits()
- SparkContextSuite.isTaskStarted = false
- SparkContextSuite.cancelStage = (cancelWhat == "stage")
- SparkContextSuite.cancelJob = (cancelWhat == "job")
+ sc.addSparkListener(listener)
val ex = intercept[SparkException] {
- sc.range(0, 10000L, numSlices = slices).mapPartitions { x =>
- SparkContextSuite.isTaskStarted = true
- // Block waiting for the listener to cancel the stage or job.
- SparkContextSuite.semaphore.acquire()
+ sc.range(0, 10000L, numSlices = 10).mapPartitions { x =>
+ x.synchronized {
+ x.wait()
+ }
x
}.count()
}
@@ -550,9 +541,11 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.")
}
+ latch.await(20, TimeUnit.SECONDS)
eventually(timeout(20.seconds)) {
assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0)
}
+ sc.removeSparkListener(listener)
}
}
@@ -637,8 +630,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
}
object SparkContextSuite {
- @volatile var cancelJob = false
- @volatile var cancelStage = false
@volatile var isTaskStarted = false
@volatile var taskKilled = false
@volatile var taskSucceeded = false
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 159629825c677..9ad2e9a5e74ac 100644
--- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala
+++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala
@@ -153,6 +153,40 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio
assert(broadcast.value.sum === 10)
}
+ test("One broadcast value instance per executor") {
+ val conf = new SparkConf()
+ .setMaster("local[4]")
+ .setAppName("test")
+
+ sc = new SparkContext(conf)
+ val list = List[Int](1, 2, 3, 4)
+ val broadcast = sc.broadcast(list)
+ val instances = sc.parallelize(1 to 10)
+ .map(x => System.identityHashCode(broadcast.value))
+ .collect()
+ .toSet
+
+ assert(instances.size === 1)
+ }
+
+ test("One broadcast value instance per executor when memory is constrained") {
+ val conf = new SparkConf()
+ .setMaster("local[4]")
+ .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)
+ val instances = sc.parallelize(1 to 10)
+ .map(x => System.identityHashCode(broadcast.value))
+ .collect()
+ .toSet
+
+ assert(instances.size === 1)
+ }
+
/**
* Verify the persistence of state associated with a TorrentBroadcast in a local-cluster.
*
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 27dd435332348..e5268ca31373e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.deploy
import java.io._
import java.net.URI
import java.nio.charset.StandardCharsets
-import java.nio.file.Files
+import java.nio.file.{Files, Paths}
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
@@ -105,6 +105,9 @@ class SparkSubmitSuite
// Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x
implicit val defaultSignaler: Signaler = ThreadSignaler
+ private val emptyIvySettings = File.createTempFile("ivy", ".xml")
+ FileUtils.write(emptyIvySettings, "", StandardCharsets.UTF_8)
+
override def beforeEach() {
super.beforeEach()
System.setProperty("spark.testing", "true")
@@ -520,6 +523,7 @@ class SparkSubmitSuite
"--repositories", repo,
"--conf", "spark.ui.enabled=false",
"--conf", "spark.master.rest.enabled=false",
+ "--conf", s"spark.jars.ivySettings=${emptyIvySettings.getAbsolutePath()}",
unusedJar.toString,
"my.great.lib.MyLib", "my.great.dep.MyLib")
runSparkSubmit(args)
@@ -530,7 +534,6 @@ class SparkSubmitSuite
val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
val main = MavenCoordinate("my.great.lib", "mylib", "0.1")
val dep = MavenCoordinate("my.great.dep", "mylib", "0.1")
- // Test using "spark.jars.packages" and "spark.jars.repositories" configurations.
IvyTestUtils.withRepository(main, Some(dep.toString), None) { repo =>
val args = Seq(
"--class", JarCreationTest.getClass.getName.stripSuffix("$"),
@@ -540,6 +543,7 @@ class SparkSubmitSuite
"--conf", s"spark.jars.repositories=$repo",
"--conf", "spark.ui.enabled=false",
"--conf", "spark.master.rest.enabled=false",
+ "--conf", s"spark.jars.ivySettings=${emptyIvySettings.getAbsolutePath()}",
unusedJar.toString,
"my.great.lib.MyLib", "my.great.dep.MyLib")
runSparkSubmit(args)
@@ -550,7 +554,6 @@ class SparkSubmitSuite
// See https://gist.github.com/shivaram/3a2fecce60768a603dac for a error log
ignore("correctly builds R packages included in a jar with --packages") {
assume(RUtils.isRInstalled, "R isn't installed on this machine.")
- // Check if the SparkR package is installed
assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.")
val main = MavenCoordinate("my.great.lib", "mylib", "0.1")
val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
@@ -563,6 +566,7 @@ class SparkSubmitSuite
"--master", "local-cluster[2,1,1024]",
"--packages", main.toString,
"--repositories", repo,
+ "--conf", s"spark.jars.ivySettings=${emptyIvySettings.getAbsolutePath()}",
"--verbose",
"--conf", "spark.ui.enabled=false",
rScriptDir)
@@ -573,7 +577,6 @@ class SparkSubmitSuite
test("include an external JAR in SparkR") {
assume(RUtils.isRInstalled, "R isn't installed on this machine.")
val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
- // Check if the SparkR package is installed
assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.")
val rScriptDir =
Seq(sparkHome, "R", "pkg", "tests", "fulltests", "jarTest.R").mkString(File.separator)
@@ -606,10 +609,13 @@ class SparkSubmitSuite
}
test("resolves command line argument paths correctly") {
- val jars = "/jar1,/jar2" // --jars
- val files = "local:/file1,file2" // --files
- val archives = "file:/archive1,archive2" // --archives
- val pyFiles = "py-file1,py-file2" // --py-files
+ val dir = Utils.createTempDir()
+ val archive = Paths.get(dir.toPath.toString, "single.zip")
+ Files.createFile(archive)
+ val jars = "/jar1,/jar2"
+ val files = "local:/file1,file2"
+ val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3"
+ val pyFiles = "py-file1,py-file2"
// Test jars and files
val clArgs = Seq(
@@ -636,9 +642,10 @@ class SparkSubmitSuite
val appArgs2 = new SparkSubmitArguments(clArgs2)
val (_, _, conf2, _) = SparkSubmit.prepareSubmitEnvironment(appArgs2)
appArgs2.files should be (Utils.resolveURIs(files))
- appArgs2.archives should be (Utils.resolveURIs(archives))
+ appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3")
conf2.get("spark.yarn.dist.files") should be (Utils.resolveURIs(files))
- conf2.get("spark.yarn.dist.archives") should be (Utils.resolveURIs(archives))
+ conf2.get("spark.yarn.dist.archives") should fullyMatch regex
+ ("file:/archive1,file:.*#archive3")
// Test python files
val clArgs3 = Seq(
@@ -657,6 +664,29 @@ class SparkSubmitSuite
conf3.get(PYSPARK_PYTHON.key) should be ("python3.5")
}
+ test("ambiguous archive mapping results in error message") {
+ val dir = Utils.createTempDir()
+ val archive1 = Paths.get(dir.toPath.toString, "first.zip")
+ val archive2 = Paths.get(dir.toPath.toString, "second.zip")
+ Files.createFile(archive1)
+ Files.createFile(archive2)
+ val jars = "/jar1,/jar2"
+ val files = "local:/file1,file2"
+ val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3"
+ val pyFiles = "py-file1,py-file2"
+
+ // Test files and archives (Yarn)
+ val clArgs2 = Seq(
+ "--master", "yarn",
+ "--class", "org.SomeClass",
+ "--files", files,
+ "--archives", archives,
+ "thejar.jar"
+ )
+
+ testPrematureExit(clArgs2.toArray, "resolves ambiguously to multiple files")
+ }
+
test("resolves config paths correctly") {
val jars = "/jar1,/jar2" // spark.jars
val files = "local:/file1,file2" // spark.files / spark.yarn.dist.files
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
index eb8c203ae7751..a0f09891787e0 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
@@ -256,4 +256,19 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
assert(jarPath.indexOf("mydep") >= 0, "should find dependency")
}
}
+
+ test("SPARK-10878: test resolution files cleaned after resolving artifact") {
+ val main = new MavenCoordinate("my.great.lib", "mylib", "0.1")
+
+ IvyTestUtils.withRepository(main, None, None) { repo =>
+ val ivySettings = SparkSubmitUtils.buildIvySettings(Some(repo), Some(tempIvyPath))
+ val jarPath = SparkSubmitUtils.resolveMavenCoordinates(
+ main.toString,
+ ivySettings,
+ isTest = true)
+ val r = """.*org.apache.spark-spark-submit-parent-.*""".r
+ assert(!ivySettings.getDefaultCache.listFiles.map(_.getName)
+ .exists(r.findFirstIn(_).isDefined), "resolution files should be cleaned")
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
index bf7480d79f8a1..155564a65c607 100644
--- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
@@ -573,7 +573,8 @@ class StandaloneDynamicAllocationSuite
syncExecutors(sc)
sc.schedulerBackend match {
case b: CoarseGrainedSchedulerBackend =>
- b.killExecutors(Seq(executorId), replace = false, force)
+ b.killExecutors(Seq(executorId), adjustTargetNumExecutors = true, countFailures = false,
+ force)
case _ => fail("expected coarse grained scheduler")
}
}
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 84ee01c7f5aaf..787de59edf465 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
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hdfs.DistributedFileSystem
import org.json4s.jackson.JsonMethods._
import org.mockito.Matchers.any
-import org.mockito.Mockito.{mock, spy, verify}
+import org.mockito.Mockito.{doReturn, mock, spy, verify}
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers
import org.scalatest.concurrent.Eventually._
@@ -149,8 +149,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) {
var mergeApplicationListingCall = 0
- override protected def mergeApplicationListing(fileStatus: FileStatus): Unit = {
- super.mergeApplicationListing(fileStatus)
+ override protected def mergeApplicationListing(
+ fileStatus: FileStatus,
+ lastSeen: Long): Unit = {
+ super.mergeApplicationListing(fileStatus, lastSeen)
mergeApplicationListingCall += 1
}
}
@@ -663,6 +665,115 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
freshUI.get.ui.store.job(0)
}
+ test("clean up stale app information") {
+ val storeDir = Utils.createTempDir()
+ val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath())
+ val provider = spy(new FsHistoryProvider(conf))
+ val appId = "new1"
+
+ // Write logs for two app attempts.
+ doReturn(1L).when(provider).getNewLastScanTime()
+ val attempt1 = newLogFile(appId, Some("1"), inProgress = false)
+ writeFile(attempt1, true, None,
+ SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("1")),
+ SparkListenerJobStart(0, 1L, Nil, null),
+ SparkListenerApplicationEnd(5L)
+ )
+ val attempt2 = newLogFile(appId, Some("2"), inProgress = false)
+ writeFile(attempt2, true, None,
+ SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("2")),
+ SparkListenerJobStart(0, 1L, Nil, null),
+ SparkListenerApplicationEnd(5L)
+ )
+ updateAndCheck(provider) { list =>
+ assert(list.size === 1)
+ assert(list(0).id === appId)
+ assert(list(0).attempts.size === 2)
+ }
+
+ // Load the app's UI.
+ val ui = provider.getAppUI(appId, Some("1"))
+ assert(ui.isDefined)
+
+ // Delete the underlying log file for attempt 1 and rescan. The UI should go away, but since
+ // attempt 2 still exists, listing data should be there.
+ doReturn(2L).when(provider).getNewLastScanTime()
+ attempt1.delete()
+ updateAndCheck(provider) { list =>
+ assert(list.size === 1)
+ assert(list(0).id === appId)
+ assert(list(0).attempts.size === 1)
+ }
+ assert(!ui.get.valid)
+ assert(provider.getAppUI(appId, None) === None)
+
+ // Delete the second attempt's log file. Now everything should go away.
+ doReturn(3L).when(provider).getNewLastScanTime()
+ attempt2.delete()
+ updateAndCheck(provider) { list =>
+ assert(list.isEmpty)
+ }
+ }
+
+ test("SPARK-21571: clean up removes invalid history files") {
+ // TODO: "maxTime" becoming negative in cleanLogs() causes this test to fail, so avoid that
+ // until we figure out what's causing the problem.
+ val clock = new ManualClock(TimeUnit.DAYS.toMillis(120))
+ val conf = createTestConf().set(MAX_LOG_AGE_S.key, s"2d")
+ val provider = new FsHistoryProvider(conf, clock) {
+ override def getNewLastScanTime(): Long = clock.getTimeMillis()
+ }
+
+ // Create 0-byte size inprogress and complete files
+ var logCount = 0
+ var validLogCount = 0
+
+ val emptyInProgress = newLogFile("emptyInprogressLogFile", None, inProgress = true)
+ emptyInProgress.createNewFile()
+ emptyInProgress.setLastModified(clock.getTimeMillis())
+ logCount += 1
+
+ val slowApp = newLogFile("slowApp", None, inProgress = true)
+ slowApp.createNewFile()
+ slowApp.setLastModified(clock.getTimeMillis())
+ logCount += 1
+
+ val emptyFinished = newLogFile("emptyFinishedLogFile", None, inProgress = false)
+ emptyFinished.createNewFile()
+ emptyFinished.setLastModified(clock.getTimeMillis())
+ logCount += 1
+
+ // Create an incomplete log file, has an end record but no start record.
+ val corrupt = newLogFile("nonEmptyCorruptLogFile", None, inProgress = false)
+ writeFile(corrupt, true, None, SparkListenerApplicationEnd(0))
+ corrupt.setLastModified(clock.getTimeMillis())
+ logCount += 1
+
+ provider.checkForLogs()
+ provider.cleanLogs()
+ assert(new File(testDir.toURI).listFiles().size === logCount)
+
+ // Move the clock forward 1 day and scan the files again. They should still be there.
+ clock.advance(TimeUnit.DAYS.toMillis(1))
+ provider.checkForLogs()
+ provider.cleanLogs()
+ assert(new File(testDir.toURI).listFiles().size === logCount)
+
+ // Update the slow app to contain valid info. Code should detect the change and not clean
+ // it up.
+ writeFile(slowApp, true, None,
+ SparkListenerApplicationStart(slowApp.getName(), Some(slowApp.getName()), 1L, "test", None))
+ slowApp.setLastModified(clock.getTimeMillis())
+ validLogCount += 1
+
+ // Move the clock forward another 2 days and scan the files again. This time the cleaner should
+ // pick up the invalid files and get rid of them.
+ clock.advance(TimeUnit.DAYS.toMillis(2))
+ provider.checkForLogs()
+ provider.cleanLogs()
+ assert(new File(testDir.toURI).listFiles().size === validLogCount)
+ }
+
/**
* Asks the provider to check for logs and calls a function to perform checks on the updated
* app list. Example:
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 3738f85da5831..4c06193225368 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
@@ -48,7 +48,7 @@ import org.apache.spark.deploy.history.config._
import org.apache.spark.status.api.v1.ApplicationInfo
import org.apache.spark.status.api.v1.JobData
import org.apache.spark.ui.SparkUI
-import org.apache.spark.util.{ResetSystemProperties, Utils}
+import org.apache.spark.util.{ResetSystemProperties, ShutdownHookManager, Utils}
/**
* A collection of tests against the historyserver, including comparing responses from the json
@@ -294,6 +294,11 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
all (siteRelativeLinks) should startWith (uiRoot)
}
+ test("/version api endpoint") {
+ val response = getUrl("version")
+ assert(response.contains(SPARK_VERSION))
+ }
+
test("ajax rendered relative links are prefixed with uiRoot (spark.ui.proxyBase)") {
val uiRoot = "/testwebproxybase"
System.setProperty("spark.ui.proxyBase", uiRoot)
@@ -564,7 +569,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
assert(jobcount === getNumJobs("/jobs"))
// no need to retain the test dir now the tests complete
- logDir.deleteOnExit()
+ ShutdownHookManager.registerShutdownDeleteDir(logDir)
}
test("ui and api authorization checks") {
diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
index eeffc36070b44..2849a10a2c81e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.deploy.security
+import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.security.Credentials
@@ -110,7 +111,64 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers {
creds.getAllTokens.size should be (0)
}
+ test("SPARK-23209: obtain tokens when Hive classes are not available") {
+ // This test needs a custom class loader to hide Hive classes which are in the classpath.
+ // Because the manager code loads the Hive provider directly instead of using reflection, we
+ // need to drive the test through the custom class loader so a new copy that cannot find
+ // Hive classes is loaded.
+ val currentLoader = Thread.currentThread().getContextClassLoader()
+ val noHive = new ClassLoader() {
+ override def loadClass(name: String, resolve: Boolean): Class[_] = {
+ if (name.startsWith("org.apache.hive") || name.startsWith("org.apache.hadoop.hive")) {
+ throw new ClassNotFoundException(name)
+ }
+
+ if (name.startsWith("java") || name.startsWith("scala")) {
+ currentLoader.loadClass(name)
+ } else {
+ val classFileName = name.replaceAll("\\.", "/") + ".class"
+ val in = currentLoader.getResourceAsStream(classFileName)
+ if (in != null) {
+ val bytes = IOUtils.toByteArray(in)
+ defineClass(name, bytes, 0, bytes.length)
+ } else {
+ throw new ClassNotFoundException(name)
+ }
+ }
+ }
+ }
+
+ try {
+ Thread.currentThread().setContextClassLoader(noHive)
+ val test = noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$"))
+ test.getMethod("runTest").invoke(null)
+ } finally {
+ Thread.currentThread().setContextClassLoader(currentLoader)
+ }
+ }
+
private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = {
Set(FileSystem.get(hadoopConf))
}
}
+
+/** Test code for SPARK-23209 to avoid using too much reflection above. */
+private object NoHiveTest extends Matchers {
+
+ def runTest(): Unit = {
+ try {
+ val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(),
+ _ => Set())
+ manager.getServiceDelegationTokenProvider("hive") should be (None)
+ } catch {
+ case e: Throwable =>
+ // Throw a better exception in case the test fails, since there may be a lot of nesting.
+ var cause = e
+ while (cause.getCause() != null) {
+ cause = cause.getCause()
+ }
+ throw cause
+ }
+ }
+
+}
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 105a178f2d94e..1a7bebe2c53cd 100644
--- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
@@ -22,6 +22,7 @@ import java.lang.Thread.UncaughtExceptionHandler
import java.nio.ByteBuffer
import java.util.Properties
import java.util.concurrent.{CountDownLatch, TimeUnit}
+import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.mutable.Map
import scala.concurrent.duration._
@@ -139,7 +140,7 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
// the fetch failure. The executor should still tell the driver that the task failed due to a
// fetch failure, not a generic exception from user code.
val inputRDD = new FetchFailureThrowingRDD(sc)
- val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false)
+ val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false, interrupt = false)
val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array())
val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array()
val task = new ResultTask(
@@ -173,17 +174,48 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
}
test("SPARK-19276: OOMs correctly handled with a FetchFailure") {
+ val (failReason, uncaughtExceptionHandler) = testFetchFailureHandling(true)
+ assert(failReason.isInstanceOf[ExceptionFailure])
+ val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable])
+ verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture())
+ assert(exceptionCaptor.getAllValues.size === 1)
+ assert(exceptionCaptor.getAllValues().get(0).isInstanceOf[OutOfMemoryError])
+ }
+
+ test("SPARK-23816: interrupts are not masked by a FetchFailure") {
+ // If killing the task causes a fetch failure, we still treat it as a task that was killed,
+ // as the fetch failure could easily be caused by interrupting the thread.
+ val (failReason, _) = testFetchFailureHandling(false)
+ assert(failReason.isInstanceOf[TaskKilled])
+ }
+
+ /**
+ * Helper for testing some cases where a FetchFailure should *not* get sent back, because its
+ * superceded by another error, either an OOM or intentionally killing a task.
+ * @param oom if true, throw an OOM after the FetchFailure; else, interrupt the task after the
+ * FetchFailure
+ */
+ private def testFetchFailureHandling(
+ oom: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = {
// when there is a fatal error like an OOM, we don't do normal fetch failure handling, since it
// may be a false positive. And we should call the uncaught exception handler.
+ // SPARK-23816 also handle interrupts the same way, as killing an obsolete speculative task
+ // does not represent a real fetch failure.
val conf = new SparkConf().setMaster("local").setAppName("executor suite test")
sc = new SparkContext(conf)
val serializer = SparkEnv.get.closureSerializer.newInstance()
val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size
- // Submit a job where a fetch failure is thrown, but then there is an OOM. We should treat
- // the fetch failure as a false positive, and just do normal OOM handling.
+ // Submit a job where a fetch failure is thrown, but then there is an OOM or interrupt. We
+ // should treat the fetch failure as a false positive, and do normal OOM or interrupt handling.
val inputRDD = new FetchFailureThrowingRDD(sc)
- val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = true)
+ if (!oom) {
+ // we are trying to setup a case where a task is killed after a fetch failure -- this
+ // is just a helper to coordinate between the task thread and this thread that will
+ // kill the task
+ ExecutorSuiteHelper.latches = new ExecutorSuiteHelper()
+ }
+ val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = oom, interrupt = !oom)
val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array())
val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array()
val task = new ResultTask(
@@ -200,15 +232,8 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
val serTask = serializer.serialize(task)
val taskDescription = createFakeTaskDescription(serTask)
- val (failReason, uncaughtExceptionHandler) =
- runTaskGetFailReasonAndExceptionHandler(taskDescription)
- // make sure the task failure just looks like a OOM, not a fetch failure
- assert(failReason.isInstanceOf[ExceptionFailure])
- val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable])
- verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture())
- assert(exceptionCaptor.getAllValues.size === 1)
- assert(exceptionCaptor.getAllValues.get(0).isInstanceOf[OutOfMemoryError])
- }
+ runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom)
+ }
test("Gracefully handle error in task deserialization") {
val conf = new SparkConf
@@ -257,22 +282,39 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
}
private def runTaskAndGetFailReason(taskDescription: TaskDescription): TaskFailedReason = {
- runTaskGetFailReasonAndExceptionHandler(taskDescription)._1
+ runTaskGetFailReasonAndExceptionHandler(taskDescription, false)._1
}
private def runTaskGetFailReasonAndExceptionHandler(
- taskDescription: TaskDescription): (TaskFailedReason, UncaughtExceptionHandler) = {
+ taskDescription: TaskDescription,
+ killTask: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = {
val mockBackend = mock[ExecutorBackend]
val mockUncaughtExceptionHandler = mock[UncaughtExceptionHandler]
var executor: Executor = null
+ val timedOut = new AtomicBoolean(false)
try {
executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true,
uncaughtExceptionHandler = mockUncaughtExceptionHandler)
// the task will be launched in a dedicated worker thread
executor.launchTask(mockBackend, taskDescription)
+ if (killTask) {
+ val killingThread = new Thread("kill-task") {
+ override def run(): Unit = {
+ // wait to kill the task until it has thrown a fetch failure
+ if (ExecutorSuiteHelper.latches.latch1.await(10, TimeUnit.SECONDS)) {
+ // now we can kill the task
+ executor.killAllTasks(true, "Killed task, eg. because of speculative execution")
+ } else {
+ timedOut.set(true)
+ }
+ }
+ }
+ killingThread.start()
+ }
eventually(timeout(5.seconds), interval(10.milliseconds)) {
assert(executor.numRunningTasks === 0)
}
+ assert(!timedOut.get(), "timed out waiting to be ready to kill tasks")
} finally {
if (executor != null) {
executor.stop()
@@ -282,8 +324,9 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
val statusCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer])
orderedMock.verify(mockBackend)
.statusUpdate(meq(0L), meq(TaskState.RUNNING), statusCaptor.capture())
+ val finalState = if (killTask) TaskState.KILLED else TaskState.FAILED
orderedMock.verify(mockBackend)
- .statusUpdate(meq(0L), meq(TaskState.FAILED), statusCaptor.capture())
+ .statusUpdate(meq(0L), meq(finalState), statusCaptor.capture())
// first statusUpdate for RUNNING has empty data
assert(statusCaptor.getAllValues().get(0).remaining() === 0)
// second update is more interesting
@@ -321,7 +364,8 @@ class SimplePartition extends Partition {
class FetchFailureHidingRDD(
sc: SparkContext,
val input: FetchFailureThrowingRDD,
- throwOOM: Boolean) extends RDD[Int](input) {
+ throwOOM: Boolean,
+ interrupt: Boolean) extends RDD[Int](input) {
override def compute(split: Partition, context: TaskContext): Iterator[Int] = {
val inItr = input.compute(split, context)
try {
@@ -330,6 +374,15 @@ class FetchFailureHidingRDD(
case t: Throwable =>
if (throwOOM) {
throw new OutOfMemoryError("OOM while handling another exception")
+ } else if (interrupt) {
+ // make sure our test is setup correctly
+ assert(TaskContext.get().asInstanceOf[TaskContextImpl].fetchFailed.isDefined)
+ // signal our test is ready for the task to get killed
+ ExecutorSuiteHelper.latches.latch1.countDown()
+ // then wait for another thread in the test to kill the task -- this latch
+ // is never actually decremented, we just wait to get killed.
+ ExecutorSuiteHelper.latches.latch2.await(10, TimeUnit.SECONDS)
+ throw new IllegalStateException("timed out waiting to be interrupted")
} else {
throw new RuntimeException("User Exception that hides the original exception", t)
}
@@ -352,6 +405,11 @@ private class ExecutorSuiteHelper {
@volatile var testFailedReason: TaskFailedReason = _
}
+// helper for coordinating killing tasks
+private object ExecutorSuiteHelper {
+ var latches: ExecutorSuiteHelper = null
+}
+
private class NonDeserializableTask extends FakeTask(0, 0) with Externalizable {
def writeExternal(out: ObjectOutput): Unit = {}
def readExternal(in: ObjectInput): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
index 3b798e36b0499..2107559572d78 100644
--- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
@@ -21,11 +21,12 @@ import java.nio.ByteBuffer
import com.google.common.io.ByteStreams
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{SharedSparkContext, SparkFunSuite}
+import org.apache.spark.internal.config
import org.apache.spark.network.util.ByteArrayWritableChannel
import org.apache.spark.util.io.ChunkedByteBuffer
-class ChunkedByteBufferSuite extends SparkFunSuite {
+class ChunkedByteBufferSuite extends SparkFunSuite with SharedSparkContext {
test("no chunks") {
val emptyChunkedByteBuffer = new ChunkedByteBuffer(Array.empty[ByteBuffer])
@@ -56,6 +57,18 @@ class ChunkedByteBufferSuite extends SparkFunSuite {
assert(chunkedByteBuffer.getChunks().head.position() === 0)
}
+ test("SPARK-24107: writeFully() write buffer which is larger than bufferWriteChunkSize") {
+ try {
+ sc.conf.set(config.BUFFER_WRITE_CHUNK_SIZE, 32L * 1024L * 1024L)
+ val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(40 * 1024 * 1024)))
+ val byteArrayWritableChannel = new ByteArrayWritableChannel(chunkedByteBuffer.size.toInt)
+ chunkedByteBuffer.writeFully(byteArrayWritableChannel)
+ assert(byteArrayWritableChannel.length() === chunkedByteBuffer.size)
+ } finally {
+ sc.conf.remove(config.BUFFER_WRITE_CHUNK_SIZE)
+ }
+ }
+
test("toArray()") {
val empty = ByteBuffer.wrap(Array.empty[Byte])
val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte))
diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
index 362cd861cc248..dcf89e4f75acf 100644
--- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
+++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
@@ -29,6 +29,7 @@ object MemoryTestingUtils {
val taskMemoryManager = new TaskMemoryManager(env.memoryManager, 0)
new TaskContextImpl(
stageId = 0,
+ stageAttemptNumber = 0,
partitionId = 0,
taskAttemptId = 0,
attemptNumber = 0,
diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
index a39e0469272fe..47af5c3320dd9 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -322,8 +322,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
}
// See SPARK-22465
- test("cogroup between multiple RDD" +
- " with number of partitions similar in order of magnitude") {
+ test("cogroup between multiple RDD with number of partitions similar in order of magnitude") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
val rdd2 = sc
.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
@@ -332,6 +331,48 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
assert(joined.getNumPartitions == rdd2.getNumPartitions)
}
+ test("cogroup between multiple RDD when defaultParallelism is set without proper partitioner") {
+ assert(!sc.conf.contains("spark.default.parallelism"))
+ try {
+ sc.conf.set("spark.default.parallelism", "4")
+ val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
+ val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)), 10)
+ val joined = rdd1.cogroup(rdd2)
+ assert(joined.getNumPartitions == sc.defaultParallelism)
+ } finally {
+ sc.conf.remove("spark.default.parallelism")
+ }
+ }
+
+ test("cogroup between multiple RDD when defaultParallelism is set with proper partitioner") {
+ assert(!sc.conf.contains("spark.default.parallelism"))
+ try {
+ sc.conf.set("spark.default.parallelism", "4")
+ val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
+ val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
+ .partitionBy(new HashPartitioner(10))
+ val joined = rdd1.cogroup(rdd2)
+ assert(joined.getNumPartitions == rdd2.getNumPartitions)
+ } finally {
+ sc.conf.remove("spark.default.parallelism")
+ }
+ }
+
+ test("cogroup between multiple RDD when defaultParallelism is set; with huge number of " +
+ "partitions in upstream RDDs") {
+ assert(!sc.conf.contains("spark.default.parallelism"))
+ try {
+ sc.conf.set("spark.default.parallelism", "4")
+ val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000)
+ val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
+ .partitionBy(new HashPartitioner(10))
+ val joined = rdd1.cogroup(rdd2)
+ assert(joined.getNumPartitions == rdd2.getNumPartitions)
+ } finally {
+ sc.conf.remove("spark.default.parallelism")
+ }
+ }
+
test("rightOuterJoin") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
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 cd1b7a9e5ab18..00867ef1308a2 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -479,7 +479,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
test("blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") {
val allocationClientMock = mock[ExecutorAllocationClient]
- when(allocationClientMock.killExecutors(any(), any(), any())).thenReturn(Seq("called"))
+ when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called"))
when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] {
// To avoid a race between blacklisting and killing, it is important that the nodeBlacklist
// is updated before we ask the executor allocation client to kill all the executors
@@ -517,7 +517,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures)
- verify(allocationClientMock, never).killExecutors(any(), any(), any())
+ verify(allocationClientMock, never).killExecutors(any(), any(), any(), any())
verify(allocationClientMock, never).killExecutorsOnHost(any())
// Enable auto-kill. Blacklist an executor and make sure killExecutors is called.
@@ -533,7 +533,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures)
- verify(allocationClientMock).killExecutors(Seq("1"), true, true)
+ verify(allocationClientMock).killExecutors(Seq("1"), false, false, true)
val taskSetBlacklist3 = createTaskSetBlacklist(stageId = 1)
// Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole
@@ -545,13 +545,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist3.execToFailures)
- verify(allocationClientMock).killExecutors(Seq("2"), true, true)
+ verify(allocationClientMock).killExecutors(Seq("2"), false, false, true)
verify(allocationClientMock).killExecutorsOnHost("hostA")
}
test("fetch failure blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") {
val allocationClientMock = mock[ExecutorAllocationClient]
- when(allocationClientMock.killExecutors(any(), any(), any())).thenReturn(Seq("called"))
+ when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called"))
when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] {
// To avoid a race between blacklisting and killing, it is important that the nodeBlacklist
// is updated before we ask the executor allocation client to kill all the executors
@@ -571,16 +571,19 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
conf.set(config.BLACKLIST_KILL_ENABLED, false)
blacklist.updateBlacklistForFetchFailure("hostA", exec = "1")
- verify(allocationClientMock, never).killExecutors(any(), any(), any())
+ verify(allocationClientMock, never).killExecutors(any(), any(), any(), any())
verify(allocationClientMock, never).killExecutorsOnHost(any())
+ assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+ assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
+
// Enable auto-kill. Blacklist an executor and make sure killExecutors is called.
conf.set(config.BLACKLIST_KILL_ENABLED, true)
blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock)
clock.advance(1000)
blacklist.updateBlacklistForFetchFailure("hostA", exec = "1")
- verify(allocationClientMock).killExecutors(Seq("1"), true, true)
+ verify(allocationClientMock).killExecutors(Seq("1"), false, false, true)
verify(allocationClientMock, never).killExecutorsOnHost(any())
assert(blacklist.executorIdToBlacklistStatus.contains("1"))
@@ -589,6 +592,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty)
+ assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+ assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
// Enable external shuffle service to see if all the executors on this node will be killed.
conf.set(config.SHUFFLE_SERVICE_ENABLED, true)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index d812b5bd92c1b..8b6ec37625eec 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -2146,6 +2146,58 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assertDataStructuresEmpty()
}
+ test("Trigger mapstage's job listener in submitMissingTasks") {
+ val rdd1 = new MyRDD(sc, 2, Nil)
+ val dep1 = new ShuffleDependency(rdd1, new HashPartitioner(2))
+ val rdd2 = new MyRDD(sc, 2, List(dep1), tracker = mapOutputTracker)
+ val dep2 = new ShuffleDependency(rdd2, new HashPartitioner(2))
+
+ val listener1 = new SimpleListener
+ val listener2 = new SimpleListener
+
+ submitMapStage(dep1, listener1)
+ submitMapStage(dep2, listener2)
+
+ // Complete the stage0.
+ assert(taskSets(0).stageId === 0)
+ complete(taskSets(0), Seq(
+ (Success, makeMapStatus("hostA", rdd1.partitions.length)),
+ (Success, makeMapStatus("hostB", rdd1.partitions.length))))
+ assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet ===
+ HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+ assert(listener1.results.size === 1)
+
+ // When attempting stage1, trigger a fetch failure.
+ assert(taskSets(1).stageId === 1)
+ complete(taskSets(1), Seq(
+ (Success, makeMapStatus("hostC", rdd2.partitions.length)),
+ (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null)))
+ scheduler.resubmitFailedStages()
+ // Stage1 listener should not have a result yet
+ assert(listener2.results.size === 0)
+
+ // Speculative task succeeded in stage1.
+ runEvent(makeCompletionEvent(
+ taskSets(1).tasks(1),
+ Success,
+ makeMapStatus("hostD", rdd2.partitions.length)))
+ // stage1 listener still should not have a result, though there's no missing partitions
+ // in it. Because stage1 has been failed and is not inside `runningStages` at this moment.
+ assert(listener2.results.size === 0)
+
+ // Stage0 should now be running as task set 2; make its task succeed
+ assert(taskSets(2).stageId === 0)
+ complete(taskSets(2), Seq(
+ (Success, makeMapStatus("hostC", rdd2.partitions.length))))
+ assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet ===
+ Set(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
+
+ // After stage0 is finished, stage1 will be submitted and found there is no missing
+ // partitions in it. Then listener got triggered.
+ assert(listener2.results.size === 1)
+ assertDataStructuresEmpty()
+ }
+
/**
* In this test, we run a map stage where one of the executors fails but we still receive a
* "zombie" complete message from that executor. We want to make sure the stage is not reported
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 03b1903902491..158c9eb75f2b6 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -35,6 +35,7 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark._
import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapRedCommitProtocol, SparkHadoopWriterUtils}
import org.apache.spark.rdd.{FakeOutputCommitter, RDD}
+import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util.{ThreadUtils, Utils}
/**
@@ -153,7 +154,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
test("Job should not complete if all commits are denied") {
// Create a mock OutputCommitCoordinator that denies all attempts to commit
doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit(
- Matchers.any(), Matchers.any(), Matchers.any())
+ Matchers.any(), Matchers.any(), Matchers.any(), Matchers.any())
val rdd: RDD[Int] = sc.parallelize(Seq(1), 1)
def resultHandler(x: Int, y: Unit): Unit = {}
val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd,
@@ -169,45 +170,106 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
test("Only authorized committer failures can clear the authorized committer lock (SPARK-6614)") {
val stage: Int = 1
+ val stageAttempt: Int = 1
val partition: Int = 2
val authorizedCommitter: Int = 3
val nonAuthorizedCommitter: Int = 100
outputCommitCoordinator.stageStart(stage, maxPartitionId = 2)
- assert(outputCommitCoordinator.canCommit(stage, partition, authorizedCommitter))
- assert(!outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter))
+ assert(outputCommitCoordinator.canCommit(stage, stageAttempt, partition, authorizedCommitter))
+ assert(!outputCommitCoordinator.canCommit(stage, stageAttempt, partition,
+ nonAuthorizedCommitter))
// The non-authorized committer fails
- outputCommitCoordinator.taskCompleted(
- stage, partition, attemptNumber = nonAuthorizedCommitter, reason = TaskKilled("test"))
+ outputCommitCoordinator.taskCompleted(stage, stageAttempt, partition,
+ attemptNumber = nonAuthorizedCommitter, reason = TaskKilled("test"))
// New tasks should still not be able to commit because the authorized committer has not failed
- assert(
- !outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 1))
+ assert(!outputCommitCoordinator.canCommit(stage, stageAttempt, partition,
+ nonAuthorizedCommitter + 1))
// The authorized committer now fails, clearing the lock
- outputCommitCoordinator.taskCompleted(
- stage, partition, attemptNumber = authorizedCommitter, reason = TaskKilled("test"))
+ outputCommitCoordinator.taskCompleted(stage, stageAttempt, partition,
+ attemptNumber = authorizedCommitter, reason = TaskKilled("test"))
// A new task should now be allowed to become the authorized committer
- assert(
- outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 2))
+ assert(outputCommitCoordinator.canCommit(stage, stageAttempt, partition,
+ nonAuthorizedCommitter + 2))
// There can only be one authorized committer
- assert(
- !outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 3))
- }
-
- test("Duplicate calls to canCommit from the authorized committer gets idempotent responses.") {
- val rdd = sc.parallelize(Seq(1), 1)
- sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).callCanCommitMultipleTimes _,
- 0 until rdd.partitions.size)
+ assert(!outputCommitCoordinator.canCommit(stage, stageAttempt, partition,
+ nonAuthorizedCommitter + 3))
}
test("SPARK-19631: Do not allow failed attempts to be authorized for committing") {
val stage: Int = 1
+ val stageAttempt: Int = 1
val partition: Int = 1
val failedAttempt: Int = 0
outputCommitCoordinator.stageStart(stage, maxPartitionId = 1)
- outputCommitCoordinator.taskCompleted(stage, partition, attemptNumber = failedAttempt,
+ outputCommitCoordinator.taskCompleted(stage, stageAttempt, partition,
+ attemptNumber = failedAttempt,
reason = ExecutorLostFailure("0", exitCausedByApp = true, None))
- assert(!outputCommitCoordinator.canCommit(stage, partition, failedAttempt))
- assert(outputCommitCoordinator.canCommit(stage, partition, failedAttempt + 1))
+ assert(!outputCommitCoordinator.canCommit(stage, stageAttempt, partition, failedAttempt))
+ assert(outputCommitCoordinator.canCommit(stage, stageAttempt, partition, failedAttempt + 1))
+ }
+
+ test("SPARK-24589: Differentiate tasks from different stage attempts") {
+ var stage = 1
+ val taskAttempt = 1
+ val partition = 1
+
+ outputCommitCoordinator.stageStart(stage, maxPartitionId = 1)
+ assert(outputCommitCoordinator.canCommit(stage, 1, partition, taskAttempt))
+ assert(!outputCommitCoordinator.canCommit(stage, 2, partition, taskAttempt))
+
+ // Fail the task in the first attempt, the task in the second attempt should succeed.
+ stage += 1
+ outputCommitCoordinator.stageStart(stage, maxPartitionId = 1)
+ outputCommitCoordinator.taskCompleted(stage, 1, partition, taskAttempt,
+ ExecutorLostFailure("0", exitCausedByApp = true, None))
+ assert(!outputCommitCoordinator.canCommit(stage, 1, partition, taskAttempt))
+ assert(outputCommitCoordinator.canCommit(stage, 2, partition, taskAttempt))
+
+ // Commit the 1st attempt, fail the 2nd attempt, make sure 3rd attempt cannot commit,
+ // then fail the 1st attempt and make sure the 4th one can commit again.
+ stage += 1
+ outputCommitCoordinator.stageStart(stage, maxPartitionId = 1)
+ assert(outputCommitCoordinator.canCommit(stage, 1, partition, taskAttempt))
+ outputCommitCoordinator.taskCompleted(stage, 2, partition, taskAttempt,
+ ExecutorLostFailure("0", exitCausedByApp = true, None))
+ assert(!outputCommitCoordinator.canCommit(stage, 3, partition, taskAttempt))
+ outputCommitCoordinator.taskCompleted(stage, 1, partition, taskAttempt,
+ ExecutorLostFailure("0", exitCausedByApp = true, None))
+ assert(outputCommitCoordinator.canCommit(stage, 4, partition, taskAttempt))
+ }
+
+ test("SPARK-24589: Make sure stage state is cleaned up") {
+ // Normal application without stage failures.
+ sc.parallelize(1 to 100, 100)
+ .map { i => (i % 10, i) }
+ .reduceByKey(_ + _)
+ .collect()
+
+ assert(sc.dagScheduler.outputCommitCoordinator.isEmpty)
+
+ // Force failures in a few tasks so that a stage is retried. Collect the ID of the failing
+ // stage so that we can check the state of the output committer.
+ val retriedStage = sc.parallelize(1 to 100, 10)
+ .map { i => (i % 10, i) }
+ .reduceByKey { case (_, _) =>
+ val ctx = TaskContext.get()
+ if (ctx.stageAttemptNumber() == 0) {
+ throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1, 1,
+ new Exception("Failure for test."))
+ } else {
+ ctx.stageId()
+ }
+ }
+ .collect()
+ .map { case (k, v) => v }
+ .toSet
+
+ assert(retriedStage.size === 1)
+ assert(sc.dagScheduler.outputCommitCoordinator.isEmpty)
+ verify(sc.env.outputCommitCoordinator, times(2))
+ .stageStart(Matchers.eq(retriedStage.head), Matchers.any())
+ verify(sc.env.outputCommitCoordinator).stageEnd(Matchers.eq(retriedStage.head))
}
}
@@ -243,16 +305,6 @@ private case class OutputCommitFunctions(tempDirPath: String) {
if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter)
}
- // Receiver should be idempotent for AskPermissionToCommitOutput
- def callCanCommitMultipleTimes(iter: Iterator[Int]): Unit = {
- val ctx = TaskContext.get()
- val canCommit1 = SparkEnv.get.outputCommitCoordinator
- .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber())
- val canCommit2 = SparkEnv.get.outputCommitCoordinator
- .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber())
- assert(canCommit1 && canCommit2)
- }
-
private def runCommitWithProvidedCommitter(
ctx: TaskContext,
iter: Iterator[Int],
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 1beb36afa95f0..6ffd1e84f7adb 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.scheduler
+import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
import java.util.concurrent.Semaphore
import scala.collection.JavaConverters._
@@ -48,7 +49,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
bus.metrics.metricRegistry.counter(s"queue.$SHARED_QUEUE.numDroppedEvents").getCount
}
- private def queueSize(bus: LiveListenerBus): Int = {
+ private def sharedQueueSize(bus: LiveListenerBus): Int = {
bus.metrics.metricRegistry.getGauges().get(s"queue.$SHARED_QUEUE.size").getValue()
.asInstanceOf[Int]
}
@@ -73,12 +74,11 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
val conf = new SparkConf()
val counter = new BasicJobCounter
val bus = new LiveListenerBus(conf)
- bus.addToSharedQueue(counter)
// Metrics are initially empty.
assert(bus.metrics.numEventsPosted.getCount === 0)
assert(numDroppedEvents(bus) === 0)
- assert(queueSize(bus) === 0)
+ assert(bus.queuedEvents.size === 0)
assert(eventProcessingTimeCount(bus) === 0)
// Post five events:
@@ -87,7 +87,10 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
// Five messages should be marked as received and queued, but no messages should be posted to
// listeners yet because the the listener bus hasn't been started.
assert(bus.metrics.numEventsPosted.getCount === 5)
- assert(queueSize(bus) === 5)
+ assert(bus.queuedEvents.size === 5)
+
+ // Add the counter to the bus after messages have been queued for later delivery.
+ bus.addToSharedQueue(counter)
assert(counter.count === 0)
// Starting listener bus should flush all buffered events
@@ -95,9 +98,12 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
Mockito.verify(mockMetricsSystem).registerSource(bus.metrics)
bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(counter.count === 5)
- assert(queueSize(bus) === 0)
+ assert(sharedQueueSize(bus) === 0)
assert(eventProcessingTimeCount(bus) === 5)
+ // After the bus is started, there should be no more queued events.
+ assert(bus.queuedEvents === null)
+
// After listener bus has stopped, posting events should not increment counter
bus.stop()
(1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) }
@@ -188,18 +194,18 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
// Post a message to the listener bus and wait for processing to begin:
bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded))
listenerStarted.acquire()
- assert(queueSize(bus) === 0)
+ assert(sharedQueueSize(bus) === 0)
assert(numDroppedEvents(bus) === 0)
// If we post an additional message then it should remain in the queue because the listener is
// busy processing the first event:
bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded))
- assert(queueSize(bus) === 1)
+ assert(sharedQueueSize(bus) === 1)
assert(numDroppedEvents(bus) === 0)
// The queue is now full, so any additional events posted to the listener will be dropped:
bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded))
- assert(queueSize(bus) === 1)
+ assert(sharedQueueSize(bus) === 1)
assert(numDroppedEvents(bus) === 1)
// Allow the the remaining events to be processed so we can stop the listener bus:
@@ -289,10 +295,13 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
val listener = new SaveStageAndTaskInfo
sc.addSparkListener(listener)
sc.addSparkListener(new StatsReportListener)
- // just to make sure some of the tasks take a noticeable amount of time
+ // just to make sure some of the tasks and their deserialization take a noticeable
+ // amount of time
+ val slowDeserializable = new SlowDeserializable
val w = { i: Int =>
if (i == 0) {
Thread.sleep(100)
+ slowDeserializable.use()
}
i
}
@@ -480,6 +489,48 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
assert(bus.findListenersByClass[BasicJobCounter]().isEmpty)
}
+ Seq(true, false).foreach { throwInterruptedException =>
+ val suffix = if (throwInterruptedException) "throw interrupt" else "set Thread interrupted"
+ test(s"interrupt within listener is handled correctly: $suffix") {
+ val conf = new SparkConf(false)
+ .set(LISTENER_BUS_EVENT_QUEUE_CAPACITY, 5)
+ val bus = new LiveListenerBus(conf)
+ val counter1 = new BasicJobCounter()
+ val counter2 = new BasicJobCounter()
+ val interruptingListener1 = new InterruptingListener(throwInterruptedException)
+ val interruptingListener2 = new InterruptingListener(throwInterruptedException)
+ bus.addToSharedQueue(counter1)
+ bus.addToSharedQueue(interruptingListener1)
+ bus.addToStatusQueue(counter2)
+ bus.addToEventLogQueue(interruptingListener2)
+ assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE, EVENT_LOG_QUEUE))
+ assert(bus.findListenersByClass[BasicJobCounter]().size === 2)
+ assert(bus.findListenersByClass[InterruptingListener]().size === 2)
+
+ bus.start(mockSparkContext, mockMetricsSystem)
+
+ // after we post one event, both interrupting listeners should get removed, and the
+ // event log queue should be removed
+ bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded))
+ bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+ assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE))
+ assert(bus.findListenersByClass[BasicJobCounter]().size === 2)
+ assert(bus.findListenersByClass[InterruptingListener]().size === 0)
+ assert(counter1.count === 1)
+ assert(counter2.count === 1)
+
+ // posting more events should be fine, they'll just get processed from the OK queue.
+ (0 until 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) }
+ bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+ assert(counter1.count === 6)
+ assert(counter2.count === 6)
+
+ // Make sure stopping works -- this requires putting a poison pill in all active queues, which
+ // would fail if our interrupted queue was still active, as its queue would be full.
+ bus.stop()
+ }
+ }
+
/**
* Assert that the given list of numbers has an average that is greater than zero.
*/
@@ -538,6 +589,18 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { throw new Exception }
}
+ /**
+ * A simple listener that interrupts on job end.
+ */
+ private class InterruptingListener(val throwInterruptedException: Boolean) extends SparkListener {
+ override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
+ if (throwInterruptedException) {
+ throw new InterruptedException("got interrupted")
+ } else {
+ Thread.currentThread().interrupt()
+ }
+ }
+ }
}
// These classes can't be declared inside of the SparkListenerSuite class because we don't want
@@ -578,3 +641,12 @@ private class FirehoseListenerThatAcceptsSparkConf(conf: SparkConf) extends Spar
case _ =>
}
}
+
+private class SlowDeserializable extends Externalizable {
+
+ override def writeExternal(out: ObjectOutput): Unit = { }
+
+ override def readExternal(in: ObjectInput): Unit = Thread.sleep(1)
+
+ def use(): Unit = { }
+}
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 a1d9085fa085d..aa9c36c0aaacb 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -29,6 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.metrics.source.JvmSource
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rdd.RDD
+import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util._
class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext {
@@ -158,6 +159,30 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
assert(attemptIdsWithFailedTask.toSet === Set(0, 1))
}
+ test("TaskContext.stageAttemptNumber getter") {
+ sc = new SparkContext("local[1,2]", "test")
+
+ // Check stageAttemptNumbers are 0 for initial stage
+ val stageAttemptNumbers = sc.parallelize(Seq(1, 2), 2).mapPartitions { _ =>
+ Seq(TaskContext.get().stageAttemptNumber()).iterator
+ }.collect()
+ assert(stageAttemptNumbers.toSet === Set(0))
+
+ // Check stageAttemptNumbers that are resubmitted when tasks have FetchFailedException
+ val stageAttemptNumbersWithFailedStage =
+ sc.parallelize(Seq(1, 2, 3, 4), 4).repartition(1).mapPartitions { _ =>
+ val stageAttemptNumber = TaskContext.get().stageAttemptNumber()
+ if (stageAttemptNumber < 2) {
+ // Throw FetchFailedException to explicitly trigger stage resubmission. A normal exception
+ // will only trigger task resubmission in the same stage.
+ throw new FetchFailedException(null, 0, 0, 0, "Fake")
+ }
+ Seq(stageAttemptNumber).iterator
+ }.collect()
+
+ assert(stageAttemptNumbersWithFailedStage.toSet === Set(2))
+ }
+
test("accumulators are updated on exception failures") {
// This means use 1 core and 4 max task failures
sc = new SparkContext("local[1,4]", "test")
@@ -190,7 +215,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
// accumulator updates from it.
val taskMetrics = TaskMetrics.empty
val task = new Task[Int](0, 0, 0) {
- context = new TaskContextImpl(0, 0, 0L, 0,
+ context = new TaskContextImpl(0, 0, 0, 0L, 0,
new TaskMemoryManager(SparkEnv.get.memoryManager, 0L),
new Properties,
SparkEnv.get.metricsSystem,
@@ -213,7 +238,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
// accumulator updates from it.
val taskMetrics = TaskMetrics.registered
val task = new Task[Int](0, 0, 0) {
- context = new TaskContextImpl(0, 0, 0L, 0,
+ context = new TaskContextImpl(0, 0, 0, 0L, 0,
new TaskMemoryManager(SparkEnv.get.memoryManager, 0L),
new Properties,
SparkEnv.get.metricsSystem,
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 6003899bb7bef..33f2ea1c94e75 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
@@ -917,4 +917,108 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
taskScheduler.initialize(new FakeSchedulerBackend)
}
}
+
+ test("Completions in zombie tasksets update status of non-zombie taskset") {
+ val taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
+ val valueSer = SparkEnv.get.serializer.newInstance()
+
+ def completeTaskSuccessfully(tsm: TaskSetManager, partition: Int): Unit = {
+ val indexInTsm = tsm.partitionToIndex(partition)
+ val matchingTaskInfo = tsm.taskAttempts.flatten.filter(_.index == indexInTsm).head
+ val result = new DirectTaskResult[Int](valueSer.serialize(1), Seq())
+ tsm.handleSuccessfulTask(matchingTaskInfo.taskId, result)
+ }
+
+ // Submit a task set, have it fail with a fetch failed, and then re-submit the task attempt,
+ // two times, so we have three active task sets for one stage. (For this to really happen,
+ // you'd need the previous stage to also get restarted, and then succeed, in between each
+ // attempt, but that happens outside what we're mocking here.)
+ val zombieAttempts = (0 until 2).map { stageAttempt =>
+ val attempt = FakeTask.createTaskSet(10, stageAttemptId = stageAttempt)
+ taskScheduler.submitTasks(attempt)
+ val tsm = taskScheduler.taskSetManagerForAttempt(0, stageAttempt).get
+ val offers = (0 until 10).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }
+ taskScheduler.resourceOffers(offers)
+ assert(tsm.runningTasks === 10)
+ // fail attempt
+ tsm.handleFailedTask(tsm.taskAttempts.head.head.taskId, TaskState.FAILED,
+ FetchFailed(null, 0, 0, 0, "fetch failed"))
+ // the attempt is a zombie, but the tasks are still running (this could be true even if
+ // we actively killed those tasks, as killing is best-effort)
+ assert(tsm.isZombie)
+ assert(tsm.runningTasks === 9)
+ tsm
+ }
+
+ // we've now got 2 zombie attempts, each with 9 tasks still active. Submit the 3rd attempt for
+ // the stage, but this time with insufficient resources so not all tasks are active.
+
+ val finalAttempt = FakeTask.createTaskSet(10, stageAttemptId = 2)
+ taskScheduler.submitTasks(finalAttempt)
+ val finalTsm = taskScheduler.taskSetManagerForAttempt(0, 2).get
+ val offers = (0 until 5).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }
+ val finalAttemptLaunchedPartitions = taskScheduler.resourceOffers(offers).flatten.map { task =>
+ finalAttempt.tasks(task.index).partitionId
+ }.toSet
+ assert(finalTsm.runningTasks === 5)
+ assert(!finalTsm.isZombie)
+
+ // We simulate late completions from our zombie tasksets, corresponding to all the pending
+ // partitions in our final attempt. This means we're only waiting on the tasks we've already
+ // launched.
+ val finalAttemptPendingPartitions = (0 until 10).toSet.diff(finalAttemptLaunchedPartitions)
+ finalAttemptPendingPartitions.foreach { partition =>
+ completeTaskSuccessfully(zombieAttempts(0), partition)
+ }
+
+ // If there is another resource offer, we shouldn't run anything. Though our final attempt
+ // used to have pending tasks, now those tasks have been completed by zombie attempts. The
+ // remaining tasks to compute are already active in the non-zombie attempt.
+ assert(
+ taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec-1", "host-1", 1))).flatten.isEmpty)
+
+ val remainingTasks = finalAttemptLaunchedPartitions.toIndexedSeq.sorted
+
+ // finally, if we finish the remaining partitions from a mix of tasksets, all attempts should be
+ // marked as zombie.
+ // for each of the remaining tasks, find the tasksets with an active copy of the task, and
+ // finish the task.
+ remainingTasks.foreach { partition =>
+ val tsm = if (partition == 0) {
+ // we failed this task on both zombie attempts, this one is only present in the latest
+ // taskset
+ finalTsm
+ } else {
+ // should be active in every taskset. We choose a zombie taskset just to make sure that
+ // we transition the active taskset correctly even if the final completion comes
+ // from a zombie.
+ zombieAttempts(partition % 2)
+ }
+ completeTaskSuccessfully(tsm, partition)
+ }
+
+ assert(finalTsm.isZombie)
+
+ // no taskset has completed all of its tasks, so no updates to the blacklist tracker yet
+ verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), anyObject())
+
+ // finally, lets complete all the tasks. We simulate failures in attempt 1, but everything
+ // else succeeds, to make sure we get the right updates to the blacklist in all cases.
+ (zombieAttempts ++ Seq(finalTsm)).foreach { tsm =>
+ val stageAttempt = tsm.taskSet.stageAttemptId
+ tsm.runningTasksSet.foreach { index =>
+ if (stageAttempt == 1) {
+ tsm.handleFailedTask(tsm.taskInfos(index).taskId, TaskState.FAILED, TaskResultLost)
+ } else {
+ val result = new DirectTaskResult[Int](valueSer.serialize(1), Seq())
+ tsm.handleSuccessfulTask(tsm.taskInfos(index).taskId, result)
+ }
+ }
+
+ // we update the blacklist for the stage attempts with all successful tasks. Even though
+ // some tasksets had failures, we still consider them all successful from a blacklisting
+ // perspective, as the failures weren't from a problem w/ the tasks themselves.
+ verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), anyObject())
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala b/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala
new file mode 100644
index 0000000000000..e57cb701b6284
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala
@@ -0,0 +1,97 @@
+/*
+ * 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.security
+
+import java.io.Closeable
+import java.net._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.internal.config._
+import org.apache.spark.util.Utils
+
+class SocketAuthHelperSuite extends SparkFunSuite {
+
+ private val conf = new SparkConf()
+ private val authHelper = new SocketAuthHelper(conf)
+
+ test("successful auth") {
+ Utils.tryWithResource(new ServerThread()) { server =>
+ Utils.tryWithResource(server.createClient()) { client =>
+ authHelper.authToServer(client)
+ server.close()
+ server.join()
+ assert(server.error == null)
+ assert(server.authenticated)
+ }
+ }
+ }
+
+ test("failed auth") {
+ Utils.tryWithResource(new ServerThread()) { server =>
+ Utils.tryWithResource(server.createClient()) { client =>
+ val badHelper = new SocketAuthHelper(new SparkConf().set(AUTH_SECRET_BIT_LENGTH, 128))
+ intercept[IllegalArgumentException] {
+ badHelper.authToServer(client)
+ }
+ server.close()
+ server.join()
+ assert(server.error != null)
+ assert(!server.authenticated)
+ }
+ }
+ }
+
+ private class ServerThread extends Thread with Closeable {
+
+ private val ss = new ServerSocket()
+ ss.bind(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0))
+
+ @volatile var error: Exception = _
+ @volatile var authenticated = false
+
+ setDaemon(true)
+ start()
+
+ def createClient(): Socket = {
+ new Socket(InetAddress.getLoopbackAddress(), ss.getLocalPort())
+ }
+
+ override def run(): Unit = {
+ var clientConn: Socket = null
+ try {
+ clientConn = ss.accept()
+ authHelper.authClient(clientConn)
+ authenticated = true
+ } catch {
+ case e: Exception =>
+ error = e
+ } finally {
+ Option(clientConn).foreach(_.close())
+ }
+ }
+
+ override def close(): Unit = {
+ try {
+ ss.close()
+ } finally {
+ interrupt()
+ }
+ }
+
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index 997c7de8dd02b..eb03ef3b3b5e3 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -195,7 +195,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
val s1Tasks = createTasks(4, execIds)
s1Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId,
+ stages.head.attemptNumber,
+ task))
}
assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size)
@@ -211,55 +213,53 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
s1Tasks.foreach { task =>
check[TaskDataWrapper](task.taskId) { wrapper =>
- assert(wrapper.info.taskId === task.taskId)
+ assert(wrapper.taskId === task.taskId)
assert(wrapper.stageId === stages.head.stageId)
assert(wrapper.stageAttemptId === stages.head.attemptId)
- assert(Arrays.equals(wrapper.stage, Array(stages.head.stageId, stages.head.attemptId)))
-
- val runtime = Array[AnyRef](stages.head.stageId: JInteger, stages.head.attemptId: JInteger,
- -1L: JLong)
- assert(Arrays.equals(wrapper.runtime, runtime))
-
- assert(wrapper.info.index === task.index)
- assert(wrapper.info.attempt === task.attemptNumber)
- assert(wrapper.info.launchTime === new Date(task.launchTime))
- assert(wrapper.info.executorId === task.executorId)
- assert(wrapper.info.host === task.host)
- assert(wrapper.info.status === task.status)
- assert(wrapper.info.taskLocality === task.taskLocality.toString())
- assert(wrapper.info.speculative === task.speculative)
+ assert(wrapper.index === task.index)
+ assert(wrapper.attempt === task.attemptNumber)
+ assert(wrapper.launchTime === task.launchTime)
+ assert(wrapper.executorId === task.executorId)
+ assert(wrapper.host === task.host)
+ assert(wrapper.status === task.status)
+ assert(wrapper.taskLocality === task.taskLocality.toString())
+ assert(wrapper.speculative === task.speculative)
}
}
- // Send executor metrics update. Only update one metric to avoid a lot of boilerplate code.
- s1Tasks.foreach { task =>
- val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED),
- Some(1L), None, true, false, None)
- listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(
- task.executorId,
- Seq((task.taskId, stages.head.stageId, stages.head.attemptId, Seq(accum)))))
- }
+ // Send two executor metrics update. Only update one metric to avoid a lot of boilerplate code.
+ // The tasks are distributed among the two executors, so the executor-level metrics should
+ // hold half of the cummulative value of the metric being updated.
+ Seq(1L, 2L).foreach { value =>
+ s1Tasks.foreach { task =>
+ val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED),
+ Some(value), None, true, false, None)
+ listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(
+ task.executorId,
+ Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum)))))
+ }
- check[StageDataWrapper](key(stages.head)) { stage =>
- assert(stage.info.memoryBytesSpilled === s1Tasks.size)
- }
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.memoryBytesSpilled === s1Tasks.size * value)
+ }
- val execs = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
- .first(key(stages.head)).last(key(stages.head)).asScala.toSeq
- assert(execs.size > 0)
- execs.foreach { exec =>
- assert(exec.info.memoryBytesSpilled === s1Tasks.size / 2)
+ val execs = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
+ .first(key(stages.head)).last(key(stages.head)).asScala.toSeq
+ assert(execs.size > 0)
+ execs.foreach { exec =>
+ assert(exec.info.memoryBytesSpilled === s1Tasks.size * value / 2)
+ }
}
// Fail one of the tasks, re-start it.
time += 1
s1Tasks.head.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
"taskType", TaskResultLost, s1Tasks.head, null))
time += 1
val reattempt = newAttempt(s1Tasks.head, nextTaskId())
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
reattempt))
assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size + 1)
@@ -275,13 +275,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
check[TaskDataWrapper](s1Tasks.head.taskId) { task =>
- assert(task.info.status === s1Tasks.head.status)
- assert(task.info.errorMessage == Some(TaskResultLost.toErrorString))
+ assert(task.status === s1Tasks.head.status)
+ assert(task.errorMessage == Some(TaskResultLost.toErrorString))
}
check[TaskDataWrapper](reattempt.taskId) { task =>
- assert(task.info.index === s1Tasks.head.index)
- assert(task.info.attempt === reattempt.attemptNumber)
+ assert(task.index === s1Tasks.head.index)
+ assert(task.attempt === reattempt.attemptNumber)
}
// Kill one task, restart it.
@@ -289,7 +289,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
val killed = s1Tasks.drop(1).head
killed.finishTime = time
killed.failed = true
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
"taskType", TaskKilled("killed"), killed, null))
check[JobDataWrapper](1) { job =>
@@ -303,21 +303,21 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
check[TaskDataWrapper](killed.taskId) { task =>
- assert(task.info.index === killed.index)
- assert(task.info.errorMessage === Some("killed"))
+ assert(task.index === killed.index)
+ assert(task.errorMessage === Some("killed"))
}
// Start a new attempt and finish it with TaskCommitDenied, make sure it's handled like a kill.
time += 1
val denied = newAttempt(killed, nextTaskId())
val denyReason = TaskCommitDenied(1, 1, 1)
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
denied))
time += 1
denied.finishTime = time
denied.failed = true
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
"taskType", denyReason, denied, null))
check[JobDataWrapper](1) { job =>
@@ -331,13 +331,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
check[TaskDataWrapper](denied.taskId) { task =>
- assert(task.info.index === killed.index)
- assert(task.info.errorMessage === Some(denyReason.toErrorString))
+ assert(task.index === killed.index)
+ assert(task.errorMessage === Some(denyReason.toErrorString))
}
// Start a new attempt.
val reattempt2 = newAttempt(denied, nextTaskId())
- listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber,
reattempt2))
// Succeed all tasks in stage 1.
@@ -350,7 +350,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
time += 1
pending.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber,
"taskType", Success, task, s1Metrics))
}
@@ -370,10 +370,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
pending.foreach { task =>
check[TaskDataWrapper](task.taskId) { wrapper =>
- assert(wrapper.info.errorMessage === None)
- assert(wrapper.info.taskMetrics.get.executorCpuTime === 2L)
- assert(wrapper.info.taskMetrics.get.executorRunTime === 4L)
- assert(wrapper.info.duration === Some(task.duration))
+ assert(wrapper.errorMessage === None)
+ assert(wrapper.executorCpuTime === 2L)
+ assert(wrapper.executorRunTime === 4L)
+ assert(wrapper.duration === task.duration)
}
}
@@ -414,13 +414,15 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
time += 1
val s2Tasks = createTasks(4, execIds)
s2Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId, stages.last.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId,
+ stages.last.attemptNumber,
+ task))
}
time += 1
s2Tasks.foreach { task =>
task.markFinished(TaskState.FAILED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber,
"taskType", TaskResultLost, task, null))
}
@@ -455,7 +457,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
// - Re-submit stage 2, all tasks, and succeed them and the stage.
val oldS2 = stages.last
- val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptId + 1, oldS2.name, oldS2.numTasks,
+ val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks,
oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics)
time += 1
@@ -466,14 +468,14 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
val newS2Tasks = createTasks(4, execIds)
newS2Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptNumber, task))
}
time += 1
newS2Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptId, "taskType", Success,
- task, null))
+ listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptNumber, "taskType",
+ Success, task, null))
}
time += 1
@@ -522,14 +524,15 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
val j2s2Tasks = createTasks(4, execIds)
j2s2Tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(j2Stages.last.stageId, j2Stages.last.attemptId,
+ listener.onTaskStart(SparkListenerTaskStart(j2Stages.last.stageId,
+ j2Stages.last.attemptNumber,
task))
}
time += 1
j2s2Tasks.foreach { task =>
task.markFinished(TaskState.FINISHED, time)
- listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptId,
+ listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptNumber,
"taskType", Success, task, null))
}
@@ -888,6 +891,27 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
assert(store.count(classOf[StageDataWrapper]) === 3)
assert(store.count(classOf[RDDOperationGraphWrapper]) === 3)
+ val dropped = stages.drop(1).head
+
+ // Cache some quantiles by calling AppStatusStore.taskSummary(). For quantiles to be
+ // calculated, we need at least one finished task. The code in AppStatusStore uses
+ // `executorRunTime` to detect valid tasks, so that metric needs to be updated in the
+ // task end event.
+ time += 1
+ val task = createTasks(1, Array("1")).head
+ listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
+
+ time += 1
+ task.markFinished(TaskState.FINISHED, time)
+ val metrics = TaskMetrics.empty
+ metrics.setExecutorRunTime(42L)
+ listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
+ "taskType", Success, task, metrics))
+
+ new AppStatusStore(store)
+ .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
+ assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 3)
+
stages.drop(1).foreach { s =>
time += 1
s.completionTime = Some(time)
@@ -899,6 +923,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
intercept[NoSuchElementException] {
store.read(classOf[StageDataWrapper], Array(2, 0))
}
+ assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 0)
val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3")
time += 1
@@ -919,13 +944,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
time += 1
val tasks = createTasks(2, Array("1"))
tasks.foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
// Start a 3rd task. The finished tasks should be deleted.
createTasks(1, Array("1")).foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
intercept[NoSuchElementException] {
@@ -934,7 +959,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
// Start a 4th task. The first task should be deleted, even if it's still running.
createTasks(1, Array("1")).foreach { task =>
- listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptNumber, task))
}
assert(store.count(classOf[TaskDataWrapper]) === 2)
intercept[NoSuchElementException] {
@@ -942,6 +967,165 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
}
+ test("eviction should respect job completion time") {
+ val testConf = conf.clone().set(MAX_RETAINED_JOBS, 2)
+ val listener = new AppStatusListener(store, testConf, true)
+
+ // Start job 1 and job 2
+ time += 1
+ listener.onJobStart(SparkListenerJobStart(1, time, Nil, null))
+ time += 1
+ listener.onJobStart(SparkListenerJobStart(2, time, Nil, null))
+
+ // Stop job 2 before job 1
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded))
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded))
+
+ // Start job 3 and job 2 should be evicted.
+ time += 1
+ listener.onJobStart(SparkListenerJobStart(3, time, Nil, null))
+ assert(store.count(classOf[JobDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[JobDataWrapper], 2)
+ }
+ }
+
+ test("eviction should respect stage completion time") {
+ val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2)
+ val listener = new AppStatusListener(store, testConf, true)
+
+ val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
+ val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2")
+ val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")
+
+ // Start stage 1 and stage 2
+ time += 1
+ stage1.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
+ time += 1
+ stage2.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage2, new Properties()))
+
+ // Stop stage 2 before stage 1
+ time += 1
+ stage2.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage2))
+ time += 1
+ stage1.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage1))
+
+ // Start stage 3 and stage 2 should be evicted.
+ stage3.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties()))
+ assert(store.count(classOf[StageDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[StageDataWrapper], Array(2, 0))
+ }
+ }
+
+ test("skipped stages should be evicted before completed stages") {
+ val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2)
+ val listener = new AppStatusListener(store, testConf, true)
+
+ val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
+ val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2")
+
+ // Sart job 1
+ time += 1
+ listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage1, stage2), null))
+
+ // Start and stop stage 1
+ time += 1
+ stage1.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
+
+ time += 1
+ stage1.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage1))
+
+ // Stop job 1 and stage 2 will become SKIPPED
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded))
+
+ // Submit stage 3 and verify stage 2 is evicted
+ val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")
+ time += 1
+ stage3.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties()))
+
+ assert(store.count(classOf[StageDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[StageDataWrapper], Array(2, 0))
+ }
+ }
+
+ test("eviction should respect task completion time") {
+ val testConf = conf.clone().set(MAX_RETAINED_TASKS_PER_STAGE, 2)
+ val listener = new AppStatusListener(store, testConf, true)
+
+ val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
+ stage1.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
+
+ // Start task 1 and task 2
+ val tasks = createTasks(3, Array("1"))
+ tasks.take(2).foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, task))
+ }
+
+ // Stop task 2 before task 1
+ time += 1
+ tasks(1).markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(
+ SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
+ time += 1
+ tasks(0).markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(
+ SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
+
+ // Start task 3 and task 2 should be evicted.
+ listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, tasks(2)))
+ assert(store.count(classOf[TaskDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[TaskDataWrapper], tasks(1).id)
+ }
+ }
+
+ test("lastStageAttempt should fail when the stage doesn't exist") {
+ val testConf = conf.clone().set(MAX_RETAINED_STAGES, 1)
+ val listener = new AppStatusListener(store, testConf, true)
+ val appStore = new AppStatusStore(store)
+
+ val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
+ val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2")
+ val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")
+
+ time += 1
+ stage1.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
+ stage1.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage1))
+
+ // Make stage 3 complete before stage 2 so that stage 3 will be evicted
+ time += 1
+ stage3.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties()))
+ stage3.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage3))
+
+ time += 1
+ stage2.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage2, new Properties()))
+ stage2.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stage2))
+
+ assert(appStore.asOption(appStore.lastStageAttempt(1)) === None)
+ assert(appStore.asOption(appStore.lastStageAttempt(2)).map(_.stageId) === Some(2))
+ assert(appStore.asOption(appStore.lastStageAttempt(3)) === None)
+ }
+
test("driver logs") {
val listener = new AppStatusListener(store, conf, true)
@@ -960,7 +1144,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
}
- private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptId)
+ private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber)
private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = {
val value = store.read(classTag[T].runtimeClass, key).asInstanceOf[T]
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
new file mode 100644
index 0000000000000..92f90f3d96ddf
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.status.api.v1.TaskMetricDistributions
+import org.apache.spark.util.Distribution
+import org.apache.spark.util.kvstore._
+
+class AppStatusStoreSuite extends SparkFunSuite {
+
+ private val uiQuantiles = Array(0.0, 0.25, 0.5, 0.75, 1.0)
+ private val stageId = 1
+ private val attemptId = 1
+
+ test("quantile calculation: 1 task") {
+ compareQuantiles(1, uiQuantiles)
+ }
+
+ test("quantile calculation: few tasks") {
+ compareQuantiles(4, uiQuantiles)
+ }
+
+ test("quantile calculation: more tasks") {
+ compareQuantiles(100, uiQuantiles)
+ }
+
+ test("quantile calculation: lots of tasks") {
+ compareQuantiles(4096, uiQuantiles)
+ }
+
+ test("quantile calculation: custom quantiles") {
+ compareQuantiles(4096, Array(0.01, 0.33, 0.5, 0.42, 0.69, 0.99))
+ }
+
+ test("quantile cache") {
+ val store = new InMemoryStore()
+ (0 until 4096).foreach { i => store.write(newTaskData(i)) }
+
+ val appStore = new AppStatusStore(store)
+
+ appStore.taskSummary(stageId, attemptId, Array(0.13d))
+ intercept[NoSuchElementException] {
+ store.read(classOf[CachedQuantile], Array(stageId, attemptId, "13"))
+ }
+
+ appStore.taskSummary(stageId, attemptId, Array(0.25d))
+ val d1 = store.read(classOf[CachedQuantile], Array(stageId, attemptId, "25"))
+
+ // Add a new task to force the cached quantile to be evicted, and make sure it's updated.
+ store.write(newTaskData(4096))
+ appStore.taskSummary(stageId, attemptId, Array(0.25d, 0.50d, 0.73d))
+
+ val d2 = store.read(classOf[CachedQuantile], Array(stageId, attemptId, "25"))
+ assert(d1.taskCount != d2.taskCount)
+
+ store.read(classOf[CachedQuantile], Array(stageId, attemptId, "50"))
+ intercept[NoSuchElementException] {
+ store.read(classOf[CachedQuantile], Array(stageId, attemptId, "73"))
+ }
+
+ assert(store.count(classOf[CachedQuantile]) === 2)
+ }
+
+ private def compareQuantiles(count: Int, quantiles: Array[Double]): Unit = {
+ val store = new InMemoryStore()
+ val values = (0 until count).map { i =>
+ val task = newTaskData(i)
+ store.write(task)
+ i.toDouble
+ }.toArray
+
+ val summary = new AppStatusStore(store).taskSummary(stageId, attemptId, quantiles).get
+ val dist = new Distribution(values, 0, values.length).getQuantiles(quantiles.sorted)
+
+ dist.zip(summary.executorRunTime).foreach { case (expected, actual) =>
+ assert(expected === actual)
+ }
+ }
+
+ private def newTaskData(i: Int): TaskDataWrapper = {
+ new TaskDataWrapper(
+ i, i, i, i, i, i, i.toString, i.toString, i.toString, i.toString, false, Nil, None,
+ i, i, i, i, i, i, i, i, i, i,
+ i, i, i, i, i, i, i, i, i, i,
+ i, i, i, i, stageId, attemptId)
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala
new file mode 100644
index 0000000000000..9e74e86ad54b9
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status
+
+import java.util.Date
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.status.api.v1.{TaskData, TaskMetrics}
+
+class AppStatusUtilsSuite extends SparkFunSuite {
+
+ test("schedulerDelay") {
+ val runningTask = new TaskData(
+ taskId = 0,
+ index = 0,
+ attempt = 0,
+ launchTime = new Date(1L),
+ resultFetchStart = None,
+ duration = Some(100L),
+ executorId = "1",
+ host = "localhost",
+ status = "RUNNING",
+ taskLocality = "PROCESS_LOCAL",
+ speculative = false,
+ accumulatorUpdates = Nil,
+ errorMessage = None,
+ taskMetrics = Some(new TaskMetrics(
+ executorDeserializeTime = 0L,
+ executorDeserializeCpuTime = 0L,
+ executorRunTime = 0L,
+ executorCpuTime = 0L,
+ resultSize = 0L,
+ jvmGcTime = 0L,
+ resultSerializationTime = 0L,
+ memoryBytesSpilled = 0L,
+ diskBytesSpilled = 0L,
+ peakExecutionMemory = 0L,
+ inputMetrics = null,
+ outputMetrics = null,
+ shuffleReadMetrics = null,
+ shuffleWriteMetrics = null)))
+ assert(AppStatusUtils.schedulerDelay(runningTask) === 0L)
+
+ val finishedTask = new TaskData(
+ taskId = 0,
+ index = 0,
+ attempt = 0,
+ launchTime = new Date(1L),
+ resultFetchStart = None,
+ duration = Some(100L),
+ executorId = "1",
+ host = "localhost",
+ status = "SUCCESS",
+ taskLocality = "PROCESS_LOCAL",
+ speculative = false,
+ accumulatorUpdates = Nil,
+ errorMessage = None,
+ taskMetrics = Some(new TaskMetrics(
+ executorDeserializeTime = 5L,
+ executorDeserializeCpuTime = 3L,
+ executorRunTime = 90L,
+ executorCpuTime = 10L,
+ resultSize = 100L,
+ jvmGcTime = 10L,
+ resultSerializationTime = 2L,
+ memoryBytesSpilled = 0L,
+ diskBytesSpilled = 0L,
+ peakExecutionMemory = 100L,
+ inputMetrics = null,
+ outputMetrics = null,
+ shuffleReadMetrics = null,
+ shuffleWriteMetrics = null)))
+ assert(AppStatusUtils.schedulerDelay(finishedTask) === 3L)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
index 917db766f7f11..9c0699bc981f8 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
@@ -62,7 +62,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach {
private def withTaskId[T](taskAttemptId: Long)(block: => T): T = {
try {
TaskContext.setTaskContext(
- new TaskContextImpl(0, 0, taskAttemptId, 0, null, new Properties, null))
+ new TaskContextImpl(0, 0, 0, taskAttemptId, 0, null, new Properties, null))
block
} finally {
TaskContext.unset()
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 5bfe9905ff17b..692ae3bf597e0 100644
--- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
@@ -352,6 +352,51 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
intercept[FetchFailedException] { iterator.next() }
}
+ test("big blocks are not checked for corruption") {
+ 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)
+ doReturn(10000L).when(corruptBuffer).size()
+
+ val blockManager = mock(classOf[BlockManager])
+ val localBmId = BlockManagerId("test-client", "test-client", 1)
+ doReturn(localBmId).when(blockManager).blockManagerId
+ doReturn(corruptBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0))
+ val localBlockLengths = Seq[Tuple2[BlockId, Long]](
+ ShuffleBlockId(0, 0, 0) -> corruptBuffer.size()
+ )
+
+ val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
+ val remoteBlockLengths = Seq[Tuple2[BlockId, Long]](
+ ShuffleBlockId(0, 1, 0) -> corruptBuffer.size()
+ )
+
+ val transfer = createMockTransfer(
+ Map(ShuffleBlockId(0, 0, 0) -> corruptBuffer, ShuffleBlockId(0, 1, 0) -> corruptBuffer))
+
+ val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
+ (localBmId, localBlockLengths),
+ (remoteBmId, remoteBlockLengths)
+ )
+
+ val taskContext = TaskContext.empty()
+ val iterator = new ShuffleBlockFetcherIterator(
+ taskContext,
+ transfer,
+ blockManager,
+ blocksByAddress,
+ (_, in) => new LimitedInputStream(in, 10000),
+ 2048,
+ Int.MaxValue,
+ Int.MaxValue,
+ Int.MaxValue,
+ true)
+ // Blocks should be returned without exceptions.
+ assert(Set(iterator.next()._1, iterator.next()._1) ===
+ Set(ShuffleBlockId(0, 0, 0), ShuffleBlockId(0, 1, 0)))
+ }
+
test("retry corrupt blocks (disabled)") {
val blockManager = mock(classOf[BlockManager])
val localBmId = BlockManagerId("test-client", "test-client", 1)
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 661d0d48d2f37..6044563f7dde7 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -28,22 +28,82 @@ import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler._
import org.apache.spark.status.AppStatusStore
-import org.apache.spark.ui.jobs.{StagePage, StagesTab}
+import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, StageData, StageStatus}
+import org.apache.spark.status.config._
+import org.apache.spark.ui.jobs.{ApiHelper, StagePage, StagesTab, TaskPagedTable}
class StagePageSuite extends SparkFunSuite with LocalSparkContext {
private val peakExecutionMemory = 10
+ test("ApiHelper.COLUMN_TO_INDEX should match headers of the task table") {
+ val conf = new SparkConf(false).set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
+ val statusStore = AppStatusStore.createLiveStore(conf)
+ try {
+ val stageData = new StageData(
+ status = StageStatus.ACTIVE,
+ stageId = 1,
+ attemptId = 1,
+ numTasks = 1,
+ numActiveTasks = 1,
+ numCompleteTasks = 1,
+ numFailedTasks = 1,
+ numKilledTasks = 1,
+ numCompletedIndices = 1,
+
+ executorRunTime = 1L,
+ executorCpuTime = 1L,
+ submissionTime = None,
+ firstTaskLaunchedTime = None,
+ completionTime = None,
+ failureReason = None,
+
+ inputBytes = 1L,
+ inputRecords = 1L,
+ outputBytes = 1L,
+ outputRecords = 1L,
+ shuffleReadBytes = 1L,
+ shuffleReadRecords = 1L,
+ shuffleWriteBytes = 1L,
+ shuffleWriteRecords = 1L,
+ memoryBytesSpilled = 1L,
+ diskBytesSpilled = 1L,
+
+ name = "stage1",
+ description = Some("description"),
+ details = "detail",
+ schedulingPool = "pool1",
+
+ rddIds = Seq(1),
+ accumulatorUpdates = Seq(new UIAccumulableInfo(0L, "acc", None, "value")),
+ tasks = None,
+ executorSummary = None,
+ killedTasksSummary = Map.empty
+ )
+ val taskTable = new TaskPagedTable(
+ stageData,
+ basePath = "/a/b/c",
+ currentTime = 0,
+ pageSize = 10,
+ sortColumn = "Index",
+ desc = false,
+ store = statusStore
+ )
+ val columnNames = (taskTable.headers \ "th" \ "a").map(_.child(1).text).toSet
+ assert(columnNames === ApiHelper.COLUMN_TO_INDEX.keySet)
+ } finally {
+ statusStore.close()
+ }
+ }
+
test("peak execution memory should displayed") {
- val conf = new SparkConf(false)
- val html = renderStagePage(conf).toString().toLowerCase(Locale.ROOT)
+ val html = renderStagePage().toString().toLowerCase(Locale.ROOT)
val targetString = "peak execution memory"
assert(html.contains(targetString))
}
test("SPARK-10543: peak execution memory should be per-task rather than cumulative") {
- val conf = new SparkConf(false)
- val html = renderStagePage(conf).toString().toLowerCase(Locale.ROOT)
+ val html = renderStagePage().toString().toLowerCase(Locale.ROOT)
// verify min/25/50/75/max show task value not cumulative values
assert(html.contains(s"$peakExecutionMemory.0 b | " * 5))
}
@@ -52,7 +112,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext {
* Render a stage page started with the given conf and return the HTML.
* This also runs a dummy stage to populate the page with useful content.
*/
- private def renderStagePage(conf: SparkConf): Seq[Node] = {
+ private def renderStagePage(): Seq[Node] = {
+ val conf = new SparkConf(false).set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
val statusStore = AppStatusStore.createLiveStore(conf)
val listener = statusStore.listener.get
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index 326546787ab6c..0f20eea735044 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -706,6 +706,23 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
}
}
+ test("stages page should show skipped stages") {
+ withSpark(newSparkContext()) { sc =>
+ val rdd = sc.parallelize(0 to 100, 100).repartition(10).cache()
+ rdd.count()
+ rdd.count()
+
+ eventually(timeout(5 seconds), interval(50 milliseconds)) {
+ goToUi(sc, "/stages")
+ find(id("skipped")).get.text should be("Skipped Stages (1)")
+ }
+ val stagesJson = getJson(sc.ui.get, "stages")
+ stagesJson.children.size should be (4)
+ val stagesStatus = stagesJson.children.map(_ \ "status")
+ stagesStatus.count(_ == JString(StageStatus.SKIPPED.name())) should be (1)
+ }
+ }
+
def goToUi(sc: SparkContext, path: String): Unit = {
goToUi(sc.ui.get, path)
}
diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala
index a04644d57ed88..fe0a9a471a651 100644
--- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala
+++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.util
import org.apache.spark._
+import org.apache.spark.serializer.JavaSerializer
class AccumulatorV2Suite extends SparkFunSuite {
@@ -162,4 +163,22 @@ class AccumulatorV2Suite extends SparkFunSuite {
assert(acc3.isZero)
assert(acc3.value === "")
}
+
+ test("LegacyAccumulatorWrapper with AccumulatorParam that has no equals/hashCode") {
+ class MyData(val i: Int) extends Serializable
+ val param = new AccumulatorParam[MyData] {
+ override def zero(initialValue: MyData): MyData = new MyData(0)
+ override def addInPlace(r1: MyData, r2: MyData): MyData = new MyData(r1.i + r2.i)
+ }
+
+ val acc = new LegacyAccumulatorWrapper(new MyData(0), param)
+ acc.metadata = AccumulatorMetadata(
+ AccumulatorContext.newId(),
+ Some("test"),
+ countFailedValues = false)
+ AccumulatorContext.register(acc)
+
+ val ser = new JavaSerializer(new SparkConf).newInstance()
+ ser.serialize(acc)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index eaea6b030c154..cde250ca65660 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -1167,6 +1167,22 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
Utils.checkAndGetK8sMasterUrl("k8s://foo://host:port")
}
}
+
+ object MalformedClassObject {
+ class MalformedClass
+ }
+
+ test("Safe getSimpleName") {
+ // getSimpleName on class of MalformedClass will result in error: Malformed class name
+ // Utils.getSimpleName works
+ val err = intercept[java.lang.InternalError] {
+ classOf[MalformedClassObject.MalformedClass].getSimpleName
+ }
+ assert(err.getMessage === "Malformed class name")
+
+ assert(Utils.getSimpleName(classOf[MalformedClassObject.MalformedClass]) ===
+ "UtilsSuite$MalformedClassObject$MalformedClass")
+ }
}
private class SimpleExtension
diff --git a/data/mllib/images/multi-channel/BGRA_alpha_60.png b/data/mllib/images/multi-channel/BGRA_alpha_60.png
new file mode 100644
index 0000000000000..913637cd2828a
Binary files /dev/null and b/data/mllib/images/multi-channel/BGRA_alpha_60.png differ
diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh
index c71137468054f..5faa3d3260a56 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -92,9 +92,9 @@ MVN="build/mvn --force"
# Hive-specific profiles for some builds
HIVE_PROFILES="-Phive -Phive-thriftserver"
# Profiles for publishing snapshots and release to Maven Central
-PUBLISH_PROFILES="-Pmesos -Pyarn -Pflume $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
+PUBLISH_PROFILES="-Pmesos -Pyarn -Pkubernetes -Pflume $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
# Profiles for building binary releases
-BASE_RELEASE_PROFILES="-Pmesos -Pyarn -Pflume -Psparkr"
+BASE_RELEASE_PROFILES="-Pmesos -Pyarn -Pkubernetes -Pflume -Psparkr"
# Scala 2.11 only profiles for some builds
SCALA_2_11_PROFILES="-Pkafka-0-8"
# Scala 2.12 only profiles for some builds
@@ -164,8 +164,6 @@ if [[ "$1" == "package" ]]; then
tar cvzf spark-$SPARK_VERSION.tgz spark-$SPARK_VERSION
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output spark-$SPARK_VERSION.tgz.asc \
--detach-sig spark-$SPARK_VERSION.tgz
- echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md MD5 spark-$SPARK_VERSION.tgz > \
- spark-$SPARK_VERSION.tgz.md5
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
SHA512 spark-$SPARK_VERSION.tgz > spark-$SPARK_VERSION.tgz.sha512
rm -rf spark-$SPARK_VERSION
@@ -215,9 +213,6 @@ if [[ "$1" == "package" ]]; then
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.sha512
@@ -234,9 +229,6 @@ if [[ "$1" == "package" ]]; then
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
--output $PYTHON_DIST_NAME.asc \
--detach-sig $PYTHON_DIST_NAME
- echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
- MD5 $PYTHON_DIST_NAME > \
- $PYTHON_DIST_NAME.md5
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
SHA512 $PYTHON_DIST_NAME > \
$PYTHON_DIST_NAME.sha512
@@ -247,9 +239,6 @@ if [[ "$1" == "package" ]]; then
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
--output spark-$SPARK_VERSION-bin-$NAME.tgz.asc \
--detach-sig spark-$SPARK_VERSION-bin-$NAME.tgz
- echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
- MD5 spark-$SPARK_VERSION-bin-$NAME.tgz > \
- spark-$SPARK_VERSION-bin-$NAME.tgz.md5
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
SHA512 spark-$SPARK_VERSION-bin-$NAME.tgz > \
spark-$SPARK_VERSION-bin-$NAME.tgz.sha512
diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py
index 730138195e5fe..32f6cbb29f0be 100755
--- a/dev/create-release/releaseutils.py
+++ b/dev/create-release/releaseutils.py
@@ -185,6 +185,8 @@ def get_commits(tag):
"graphx": "GraphX",
"input/output": CORE_COMPONENT,
"java api": "Java API",
+ "k8s": "Kubernetes",
+ "kubernetes": "Kubernetes",
"mesos": "Mesos",
"ml": "MLlib",
"mllib": "MLlib",
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index a7fce2ede0ea5..4f0794d6f1a11 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -17,6 +17,7 @@ arpack_combined_all-0.1.jar
arrow-format-0.8.0.jar
arrow-memory-0.8.0.jar
arrow-vector-0.8.0.jar
+automaton-1.11-8.jar
avro-1.7.7.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
@@ -60,6 +61,7 @@ datanucleus-rdbms-3.2.9.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
flatbuffers-1.2.0-3f79e055.jar
+generex-1.0.1.jar
gson-2.2.4.jar
guava-14.0.1.jar
guice-3.0.jar
@@ -91,8 +93,10 @@ jackson-annotations-2.6.7.jar
jackson-core-2.6.7.jar
jackson-core-asl-1.9.13.jar
jackson-databind-2.6.7.1.jar
+jackson-dataformat-yaml-2.6.7.jar
jackson-jaxrs-1.9.13.jar
jackson-mapper-asl-1.9.13.jar
+jackson-module-jaxb-annotations-2.6.7.jar
jackson-module-paranamer-2.7.9.jar
jackson-module-scala_2.11-2.6.7.1.jar
jackson-xc-1.9.13.jar
@@ -130,10 +134,13 @@ jta-1.1.jar
jtransforms-2.4.0.jar
jul-to-slf4j-1.7.16.jar
kryo-shaded-3.0.3.jar
+kubernetes-client-3.0.0.jar
+kubernetes-model-2.0.0.jar
leveldbjni-all-1.8.jar
libfb303-0.9.3.jar
libthrift-0.9.3.jar
log4j-1.2.17.jar
+logging-interceptor-3.8.1.jar
lz4-java-1.4.0.jar
machinist_2.11-0.6.1.jar
macro-compat_2.11-1.1.1.jar
@@ -146,21 +153,23 @@ minlog-1.3.0.jar
netty-3.9.9.Final.jar
netty-all-4.1.17.Final.jar
objenesis-2.1.jar
+okhttp-3.8.1.jar
+okio-1.13.0.jar
opencsv-2.3.jar
-orc-core-1.4.1-nohive.jar
-orc-mapreduce-1.4.1-nohive.jar
+orc-core-1.4.4-nohive.jar
+orc-mapreduce-1.4.4-nohive.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
paranamer-2.8.jar
-parquet-column-1.8.2.jar
-parquet-common-1.8.2.jar
-parquet-encoding-1.8.2.jar
+parquet-column-1.8.3.jar
+parquet-common-1.8.3.jar
+parquet-encoding-1.8.3.jar
parquet-format-2.3.1.jar
-parquet-hadoop-1.8.2.jar
+parquet-hadoop-1.8.3.jar
parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.8.2.jar
+parquet-jackson-1.8.3.jar
protobuf-java-2.5.0.jar
-py4j-0.10.6.jar
+py4j-0.10.7.jar
pyrolite-4.13.jar
scala-compiler-2.11.8.jar
scala-library-2.11.8.jar
@@ -171,6 +180,7 @@ scalap-2.11.8.jar
shapeless_2.11-2.3.2.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
+snakeyaml-1.15.jar
snappy-0.2.jar
snappy-java-1.1.2.6.jar
spire-macros_2.11-0.13.0.jar
@@ -186,5 +196,6 @@ xbean-asm5-shaded-4.4.jar
xercesImpl-2.9.1.jar
xmlenc-0.52.jar
xz-1.0.jar
+zjsonpatch-0.3.0.jar
zookeeper-3.4.6.jar
zstd-jni-1.3.2-2.jar
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index 94b2e98d85e74..df2be777ff5ac 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -17,6 +17,7 @@ arpack_combined_all-0.1.jar
arrow-format-0.8.0.jar
arrow-memory-0.8.0.jar
arrow-vector-0.8.0.jar
+automaton-1.11-8.jar
avro-1.7.7.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
@@ -60,6 +61,7 @@ datanucleus-rdbms-3.2.9.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
flatbuffers-1.2.0-3f79e055.jar
+generex-1.0.1.jar
gson-2.2.4.jar
guava-14.0.1.jar
guice-3.0.jar
@@ -91,8 +93,10 @@ jackson-annotations-2.6.7.jar
jackson-core-2.6.7.jar
jackson-core-asl-1.9.13.jar
jackson-databind-2.6.7.1.jar
+jackson-dataformat-yaml-2.6.7.jar
jackson-jaxrs-1.9.13.jar
jackson-mapper-asl-1.9.13.jar
+jackson-module-jaxb-annotations-2.6.7.jar
jackson-module-paranamer-2.7.9.jar
jackson-module-scala_2.11-2.6.7.1.jar
jackson-xc-1.9.13.jar
@@ -131,10 +135,13 @@ jta-1.1.jar
jtransforms-2.4.0.jar
jul-to-slf4j-1.7.16.jar
kryo-shaded-3.0.3.jar
+kubernetes-client-3.0.0.jar
+kubernetes-model-2.0.0.jar
leveldbjni-all-1.8.jar
libfb303-0.9.3.jar
libthrift-0.9.3.jar
log4j-1.2.17.jar
+logging-interceptor-3.8.1.jar
lz4-java-1.4.0.jar
machinist_2.11-0.6.1.jar
macro-compat_2.11-1.1.1.jar
@@ -147,21 +154,23 @@ minlog-1.3.0.jar
netty-3.9.9.Final.jar
netty-all-4.1.17.Final.jar
objenesis-2.1.jar
+okhttp-3.8.1.jar
+okio-1.13.0.jar
opencsv-2.3.jar
-orc-core-1.4.1-nohive.jar
-orc-mapreduce-1.4.1-nohive.jar
+orc-core-1.4.4-nohive.jar
+orc-mapreduce-1.4.4-nohive.jar
oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
paranamer-2.8.jar
-parquet-column-1.8.2.jar
-parquet-common-1.8.2.jar
-parquet-encoding-1.8.2.jar
+parquet-column-1.8.3.jar
+parquet-common-1.8.3.jar
+parquet-encoding-1.8.3.jar
parquet-format-2.3.1.jar
-parquet-hadoop-1.8.2.jar
+parquet-hadoop-1.8.3.jar
parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.8.2.jar
+parquet-jackson-1.8.3.jar
protobuf-java-2.5.0.jar
-py4j-0.10.6.jar
+py4j-0.10.7.jar
pyrolite-4.13.jar
scala-compiler-2.11.8.jar
scala-library-2.11.8.jar
@@ -172,6 +181,7 @@ scalap-2.11.8.jar
shapeless_2.11-2.3.2.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
+snakeyaml-1.15.jar
snappy-0.2.jar
snappy-java-1.1.2.6.jar
spire-macros_2.11-0.13.0.jar
@@ -187,5 +197,6 @@ xbean-asm5-shaded-4.4.jar
xercesImpl-2.9.1.jar
xmlenc-0.52.jar
xz-1.0.jar
+zjsonpatch-0.3.0.jar
zookeeper-3.4.6.jar
zstd-jni-1.3.2-2.jar
diff --git a/dev/lint-java b/dev/lint-java
index c2e80538ef2a5..1f0b0c8379ed0 100755
--- a/dev/lint-java
+++ b/dev/lint-java
@@ -20,7 +20,7 @@
SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)"
-ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR)
+ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR)
if test ! -z "$ERRORS"; then
echo -e "Checkstyle checks failed at following occurrences:\n$ERRORS"
diff --git a/dev/mima b/dev/mima
index 1e3ca9700bc07..cd2694ff4d3de 100755
--- a/dev/mima
+++ b/dev/mima
@@ -24,7 +24,7 @@ set -e
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
cd "$FWDIR"
-SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pyarn -Pflume -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
+SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)"
OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"
diff --git a/dev/run-pip-tests b/dev/run-pip-tests
index d51dde12a03c5..03fc83298dc2f 100755
--- a/dev/run-pip-tests
+++ b/dev/run-pip-tests
@@ -89,7 +89,7 @@ for python in "${PYTHON_EXECS[@]}"; do
source "$VIRTUALENV_PATH"/bin/activate
fi
# Upgrade pip & friends if using virutal env
- if [ ! -n "USE_CONDA" ]; then
+ if [ ! -n "$USE_CONDA" ]; then
pip install --upgrade pip pypandoc wheel numpy
fi
diff --git a/dev/scalastyle b/dev/scalastyle
index 89ecc8abd6f8c..b8053df05fa2b 100755
--- a/dev/scalastyle
+++ b/dev/scalastyle
@@ -24,6 +24,7 @@ ERRORS=$(echo -e "q\n" \
-Pkinesis-asl \
-Pmesos \
-Pkafka-0-8 \
+ -Pkubernetes \
-Pyarn \
-Pflume \
-Phive \
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index f834563da9dda..b900f0bd913c3 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -400,6 +400,7 @@ def __hash__(self):
"pyspark.sql.functions",
"pyspark.sql.readwriter",
"pyspark.sql.streaming",
+ "pyspark.sql.udf",
"pyspark.sql.window",
"pyspark.sql.tests",
]
@@ -539,7 +540,7 @@ def __hash__(self):
kubernetes = Module(
name="kubernetes",
dependencies=[],
- source_file_regexes=["resource-managers/kubernetes/core"],
+ source_file_regexes=["resource-managers/kubernetes"],
build_profile_flags=["-Pkubernetes"],
sbt_test_goals=["kubernetes/test"]
)
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index 58b295d4f6e00..3bf7618e1ea96 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -29,7 +29,7 @@ export LC_ALL=C
# TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution.
# NOTE: These should match those in the release publishing script
-HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pyarn -Pflume -Phive"
+HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Phive"
MVN="build/mvn"
HADOOP_PROFILES=(
hadoop-2.6
diff --git a/docs/README.md b/docs/README.md
index 225bb1b2040de..166a7e572982d 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -22,10 +22,13 @@ $ sudo gem install jekyll jekyll-redirect-from pygments.rb
$ sudo pip install Pygments
# Following is needed only for generating API docs
$ sudo pip install sphinx pypandoc mkdocs
-$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", "testthat", "rmarkdown"), repos="http://cran.stat.ucla.edu/")'
+$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "testthat", "rmarkdown"), repos="http://cran.stat.ucla.edu/")'
+$ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="http://cran.stat.ucla.edu/")'
```
-(Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0)
+Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0.
+
+Note: Other versions of roxygen2 might work in SparkR documentation generation but `RoxygenNote` field in `$SPARK_HOME/R/pkg/DESCRIPTION` is 5.0.1, which is updated if the version is mismatched.
## Generating the Documentation HTML
@@ -62,12 +65,12 @@ $ PRODUCTION=1 jekyll build
## API Docs (Scaladoc, Javadoc, Sphinx, roxygen2, MkDocs)
-You can build just the Spark scaladoc and javadoc by running `build/sbt unidoc` from the `SPARK_HOME` directory.
+You can build just the Spark scaladoc and javadoc by running `build/sbt unidoc` from the `$SPARK_HOME` directory.
Similarly, you can build just the PySpark docs by running `make html` from the
-`SPARK_HOME/python/docs` directory. Documentation is only generated for classes that are listed as
-public in `__init__.py`. The SparkR docs can be built by running `SPARK_HOME/R/create-docs.sh`, and
-the SQL docs can be built by running `SPARK_HOME/sql/create-docs.sh`
+`$SPARK_HOME/python/docs` directory. Documentation is only generated for classes that are listed as
+public in `__init__.py`. The SparkR docs can be built by running `$SPARK_HOME/R/create-docs.sh`, and
+the SQL docs can be built by running `$SPARK_HOME/sql/create-docs.sh`
after [building Spark](https://github.com/apache/spark#building-spark) first.
When you run `jekyll build` in the `docs` directory, it will also copy over the scaladoc and javadoc for the various
diff --git a/docs/_config.yml b/docs/_config.yml
index dcc211204d766..8579166c2e635 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.3.0-SNAPSHOT
-SPARK_VERSION_SHORT: 2.3.0
+SPARK_VERSION: 2.3.2-SNAPSHOT
+SPARK_VERSION_SHORT: 2.3.2
SCALA_BINARY_VERSION: "2.11"
SCALA_VERSION: "2.11.8"
MESOS_VERSION: 1.0.0
diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md
index 751a192da4ffd..c150d9efc06ff 100644
--- a/docs/cloud-integration.md
+++ b/docs/cloud-integration.md
@@ -180,10 +180,10 @@ under the path, not the number of *new* files, so it can become a slow operation
The size of the window needs to be set to handle this.
1. Files only appear in an object store once they are completely written; there
-is no need for a worklow of write-then-rename to ensure that files aren't picked up
+is no need for a workflow of write-then-rename to ensure that files aren't picked up
while they are still being written. Applications can write straight to the monitored directory.
-1. Streams should only be checkpointed to an store implementing a fast and
+1. Streams should only be checkpointed to a store implementing a fast and
atomic `rename()` operation Otherwise the checkpointing may be slow and potentially unreliable.
## Further Reading
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 658e67f99dd71..7277e2fb2731d 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -52,8 +52,8 @@ The system currently supports three cluster managers:
* [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
and service applications.
* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
-* [Kubernetes](running-on-kubernetes.html) -- [Kubernetes](https://kubernetes.io/docs/concepts/overview/what-is-kubernetes/)
-is an open-source platform that provides container-centric infrastructure.
+* [Kubernetes](running-on-kubernetes.html) -- an open-source system for automating deployment, scaling,
+ and management of containerized applications.
A third-party project (not supported by the Spark project) exists to add support for
[Nomad](https://github.com/hashicorp/nomad-spark) as a cluster manager.
diff --git a/docs/configuration.md b/docs/configuration.md
index 1189aea2aa71f..ec4f5d41c2d9d 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -58,6 +58,10 @@ The following format is accepted:
1t or 1tb (tebibytes = 1024 gibibytes)
1p or 1pb (pebibytes = 1024 tebibytes)
+While numbers without units are generally interpreted as bytes, a few are interpreted as KiB or MiB.
+See documentation of individual configuration properties. Specifying units is desirable where
+possible.
+
## Dynamically Loading Spark Properties
In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For
@@ -75,7 +79,7 @@ Then, you can supply configuration values at runtime:
{% endhighlight %}
The Spark shell and [`spark-submit`](submitting-applications.html)
-tool support two ways to load configurations dynamically. The first are command line options,
+tool support two ways to load configurations dynamically. The first is command line options,
such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf`
flag, but uses special flags for properties that play a part in launching the Spark application.
Running `./bin/spark-submit --help` will show the entire list of these options.
@@ -136,9 +140,9 @@ of the most common options to set are:
spark.driver.maxResultSize |
1g |
- Limit of total size of serialized results of all partitions for each Spark action (e.g. collect).
- Should be at least 1M, or 0 for unlimited. Jobs will be aborted if the total size
- is above this limit.
+ Limit of total size of serialized results of all partitions for each Spark action (e.g.
+ collect) in bytes. Should be at least 1M, or 0 for unlimited. Jobs will be aborted if the total
+ size is above this limit.
Having a high limit may cause out-of-memory errors in driver (depends on spark.driver.memory
and memory overhead of objects in JVM). Setting a proper limit can protect the driver from
out-of-memory errors.
@@ -148,10 +152,10 @@ of the most common options to set are:
| spark.driver.memory |
1g |
- Amount of memory to use for the driver process, i.e. where SparkContext is initialized.
- (e.g. 1g , 2g ).
-
- Note: In client mode, this config must not be set through the SparkConf
+ Amount of memory to use for the driver process, i.e. where SparkContext is initialized, in MiB
+ unless otherwise specified (e.g. 1g , 2g ).
+
+ Note: In client mode, this config must not be set through the SparkConf
directly in your application, because the driver JVM has already started at that point.
Instead, please set this through the --driver-memory command line option
or in your default properties file.
@@ -161,27 +165,28 @@ of the most common options to set are:
| spark.driver.memoryOverhead |
driverMemory * 0.10, with minimum of 384 |
- The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is
- memory that accounts for things like VM overheads, interned strings, other native overheads, etc.
- This tends to grow with the container size (typically 6-10%). This option is currently supported
- on YARN and Kubernetes.
+ The amount of off-heap memory to be allocated per driver in cluster mode, in MiB unless
+ otherwise specified. This is memory that accounts for things like VM overheads, interned strings,
+ other native overheads, etc. This tends to grow with the container size (typically 6-10%).
+ This option is currently supported on YARN and Kubernetes.
|
|