From 8d5765751c500de5fb699100d5b10eebbcab4e5e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 29 Dec 2015 18:27:27 -0800 Subject: [PATCH 01/65] Remove unused methods or replace them There are a bunch of decrement X methods that were not used. Also, there are a few set X methods that could have easily just been increment X. The latter change is more in line with accumulators. --- .../apache/spark/executor/TaskMetrics.scala | 39 ++++++------------- .../org/apache/spark/scheduler/Task.scala | 1 - .../org/apache/spark/util/JsonProtocol.scala | 6 +-- .../apache/spark/util/JsonProtocolSuite.scala | 5 +-- 4 files changed, 17 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 42207a955359..cc9bf2c01a6c 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,7 +17,6 @@ package org.apache.spark.executor -import java.io.{IOException, ObjectInputStream} import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer @@ -40,13 +39,12 @@ import org.apache.spark.util.Utils * shipping off at any time to consumers of the SparkListener interface. */ @DeveloperApi -class TaskMetrics extends Serializable { - /** - * Host's name the task runs on - */ - private var _hostname: String = _ - def hostname: String = _hostname - private[spark] def setHostname(value: String) = _hostname = value +class TaskMetrics(val hostname: String) extends Serializable { + + // Needed for Java + def this() { + this(TaskMetrics.getCachedHostName) + } /** * Time taken on the executor to deserialize this task @@ -91,7 +89,6 @@ class TaskMetrics extends Serializable { private var _memoryBytesSpilled: Long = _ def memoryBytesSpilled: Long = _memoryBytesSpilled private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value - private[spark] def decMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled -= value /** * The number of on-disk bytes spilled by this task @@ -99,7 +96,6 @@ class TaskMetrics extends Serializable { private var _diskBytesSpilled: Long = _ def diskBytesSpilled: Long = _diskBytesSpilled private[spark] def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value - private[spark] def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read @@ -215,15 +211,6 @@ class TaskMetrics extends Serializable { inputMetrics.foreach(_.updateBytesRead()) } - @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { - in.defaultReadObject() - // Get the hostname from cached data, since hostname is the order of number of nodes in - // cluster, so using cached hostname will decrease the object number and alleviate the GC - // overhead. - _hostname = TaskMetrics.getCachedHostName(_hostname) - } - private var _accumulatorUpdates: Map[Long, Any] = Map.empty @transient private var _accumulatorsUpdater: () => Map[Long, Any] = null @@ -246,7 +233,12 @@ private[spark] object TaskMetrics { def empty: TaskMetrics = new TaskMetrics - def getCachedHostName(host: String): String = { + /** + * Get the hostname from cached data, since hostname is the order of number of nodes in cluster, + * so using cached hostname will decrease the object number and alleviate the GC overhead. + */ + def getCachedHostName: String = { + val host = Utils.localHostName() val canonicalHost = hostNameCache.putIfAbsent(host, host) if (canonicalHost != null) canonicalHost else host } @@ -350,7 +342,6 @@ class ShuffleReadMetrics extends Serializable { private var _remoteBlocksFetched: Int = _ def remoteBlocksFetched: Int = _remoteBlocksFetched private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value - private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value /** * Number of local blocks fetched in this shuffle by this task @@ -358,7 +349,6 @@ class ShuffleReadMetrics extends Serializable { private var _localBlocksFetched: Int = _ def localBlocksFetched: Int = _localBlocksFetched private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value - private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value /** * Time the task spent waiting for remote shuffle blocks. This only includes the time @@ -368,7 +358,6 @@ class ShuffleReadMetrics extends Serializable { private var _fetchWaitTime: Long = _ def fetchWaitTime: Long = _fetchWaitTime private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value - private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value /** * Total number of remote bytes read from the shuffle by this task @@ -376,7 +365,6 @@ class ShuffleReadMetrics extends Serializable { private var _remoteBytesRead: Long = _ def remoteBytesRead: Long = _remoteBytesRead private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value - private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). @@ -401,7 +389,6 @@ class ShuffleReadMetrics extends Serializable { private var _recordsRead: Long = _ def recordsRead: Long = _recordsRead private[spark] def incRecordsRead(value: Long) = _recordsRead += value - private[spark] def decRecordsRead(value: Long) = _recordsRead -= value } /** @@ -424,7 +411,6 @@ class ShuffleWriteMetrics extends Serializable { @volatile private var _shuffleWriteTime: Long = _ def shuffleWriteTime: Long = _shuffleWriteTime private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value - private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value /** * Total number of records written to the shuffle by this task @@ -433,5 +419,4 @@ class ShuffleWriteMetrics extends Serializable { def shuffleRecordsWritten: Long = _shuffleRecordsWritten private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value - private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 9f27eed626be..e1139958997d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -78,7 +78,6 @@ private[spark] abstract class Task[T]( internalAccumulators, runningLocally = false) TaskContext.setTaskContext(context) - context.taskMetrics.setHostname(Utils.localHostName()) context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) taskThread = Thread.currentThread() if (_killed) { 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 cb0f1bf79f3d..4e8eaeeefda5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -714,8 +714,8 @@ private[spark] object JsonProtocol { if (json == JNothing) { return TaskMetrics.empty } - val metrics = new TaskMetrics - metrics.setHostname((json \ "Host Name").extract[String]) + val hostname = (json \ "Host Name").extract[String] + val metrics = new TaskMetrics(hostname) metrics.setExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long]) metrics.setExecutorRunTime((json \ "Executor Run Time").extract[Long]) metrics.setResultSize((json \ "Result Size").extract[Long]) @@ -757,7 +757,7 @@ private[spark] object JsonProtocol { val metrics = new ShuffleWriteMetrics metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) - metrics.setShuffleRecordsWritten((json \ "Shuffle Records Written") + metrics.incShuffleRecordsWritten((json \ "Shuffle Records Written") .extractOpt[Long].getOrElse(0)) metrics } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 1939ce5c743b..a8f2ce03e90b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -764,8 +764,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput: Boolean, hasOutput: Boolean, hasRecords: Boolean = true) = { - val t = new TaskMetrics - t.setHostname("localhost") + val t = new TaskMetrics("localhost") t.setExecutorDeserializeTime(a) t.setExecutorRunTime(b) t.setResultSize(c) @@ -797,7 +796,7 @@ class JsonProtocolSuite extends SparkFunSuite { val sw = new ShuffleWriteMetrics sw.incShuffleBytesWritten(a + b + c) sw.incShuffleWriteTime(b + c + d) - sw.setShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) + sw.incShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) t.shuffleWriteMetrics = Some(sw) } // Make at most 6 blocks From 1ad286813b758fda992c3d85406a7a628c39d1f7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 4 Jan 2016 13:34:42 -0800 Subject: [PATCH 02/65] Implement initial framework to migrate metrics to accums This commit uses the existing PEAK_EXECUTION_MEMORY mechanism to bring a few other fields in TaskMetrics to use accumulators. --- .../scala/org/apache/spark/Accumulators.scala | 77 +++++++++---------- .../org/apache/spark/TaskContextImpl.scala | 5 +- .../apache/spark/executor/TaskMetrics.scala | 43 ++++++++--- .../org/apache/spark/AccumulatorSuite.scala | 28 +++---- 4 files changed, 88 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5592b75afb75..4e30b39b1b36 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -64,11 +64,10 @@ class Accumulable[R, T] private[spark] ( def this(@transient initialValue: R, param: AccumulableParam[R, T]) = this(initialValue, param, None) - val id: Long = Accumulators.newId + val id: Long = Accumulators.newId() - @volatile @transient private var value_ : R = initialValue // Current value on master + @volatile private var value_ : R = initialValue val zero = param.zero(initialValue) // Zero value to be passed to workers - private var deserialized = false Accumulators.register(this) @@ -110,13 +109,7 @@ class Accumulable[R, T] private[spark] ( /** * Access the accumulator's current value; only allowed on master. */ - def value: R = { - if (!deserialized) { - value_ - } else { - throw new UnsupportedOperationException("Can't read accumulator value in task") - } - } + def value: R = value_ /** * Get the current value of this accumulator from within a task. @@ -133,11 +126,7 @@ class Accumulable[R, T] private[spark] ( * Set the accumulator's value; only allowed on master. */ def value_= (newValue: R) { - if (!deserialized) { - value_ = newValue - } else { - throw new UnsupportedOperationException("Can't assign accumulator value in task") - } + value_ = newValue } /** @@ -150,8 +139,10 @@ class Accumulable[R, T] private[spark] ( // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() - value_ = zero - deserialized = true + // TODO: fix me? + // value_ = zero + // deserialized = true + // Automatically register the accumulator when it is deserialized with the task closure. // // Note internal accumulators sent with task are deserialized before the TaskContext is created @@ -361,18 +352,33 @@ private[spark] object Accumulators extends Logging { } private[spark] object InternalAccumulator { + val EXECUTOR_DESERIALIZE_TIME = "executorDeserializeTime" + val EXECUTOR_RUN_TIME = "executorRunTime" val PEAK_EXECUTION_MEMORY = "peakExecutionMemory" - val TEST_ACCUMULATOR = "testAccumulator" - - // For testing only. - // This needs to be a def since we don't want to reuse the same accumulator across stages. - private def maybeTestAccumulator: Option[Accumulator[Long]] = { - if (sys.props.contains("spark.testing")) { - Some(new Accumulator( - 0L, AccumulatorParam.LongAccumulatorParam, Some(TEST_ACCUMULATOR), internal = true)) - } else { - None - } + val TEST_ACCUM = "testAccumulator" + + /** + * Create a new internal Long accumulator with the specified name. + */ + private def newMetric(name: String): Accumulator[Long] = { + new Accumulator(0L, AccumulatorParam.LongAccumulatorParam, Some(name), internal = true) + } + + /** + * Accumulators for tracking internal metrics. + * Note: this method does not register accumulators for clean up. + */ + def create(): Seq[Accumulator[Long]] = { + val maybeTestAccumulator = sys.props.get("spark.testing").map(_ => newMetric(TEST_ACCUM)).toSeq + Seq( + newMetric(EXECUTOR_DESERIALIZE_TIME), + newMetric(EXECUTOR_RUN_TIME), + // Execution memory refers to the memory used by internal data structures created + // during shuffles, aggregations and joins. The value of this accumulator should be + // approximately the sum of the peak sizes across all such data structures created + // in this task. For SQL jobs, this only tracks all unsafe operators and ExternalSort. + newMetric(PEAK_EXECUTION_MEMORY) + ) ++ maybeTestAccumulator } /** @@ -383,17 +389,10 @@ private[spark] object InternalAccumulator { * values across all tasks within each stage. */ def create(sc: SparkContext): Seq[Accumulator[Long]] = { - val internalAccumulators = Seq( - // Execution memory refers to the memory used by internal data structures created - // during shuffles, aggregations and joins. The value of this accumulator should be - // approximately the sum of the peak sizes across all such data structures created - // in this task. For SQL jobs, this only tracks all unsafe operators and ExternalSort. - new Accumulator( - 0L, AccumulatorParam.LongAccumulatorParam, Some(PEAK_EXECUTION_MEMORY), internal = true) - ) ++ maybeTestAccumulator.toSeq - internalAccumulators.foreach { accumulator => - sc.cleaner.foreach(_.registerAccumulatorForCleanup(accumulator)) + val accums = create() + accums.foreach { accum => + sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) } - internalAccumulators + accums } } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index f0ae83a9341b..06b6555225ee 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -33,8 +33,7 @@ private[spark] class TaskContextImpl( override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, internalAccumulators: Seq[Accumulator[Long]], - val runningLocally: Boolean = false, - val taskMetrics: TaskMetrics = TaskMetrics.empty) + val runningLocally: Boolean = false) extends TaskContext with Logging { @@ -122,4 +121,6 @@ private[spark] class TaskContextImpl( internalAccumulators.foreach(registerAccumulator) internalAccumulators.map { a => (a.name.get, a) }.toMap } + + val taskMetrics: TaskMetrics = new TaskMetrics(internalMetricsToAccumulators) } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index cc9bf2c01a6c..639d5336699c 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -21,13 +21,13 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{Accumulator, InternalAccumulator} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * Metrics tracked during the execution of a task. * * This class is used to house metrics both for in-progress and completed tasks. In executors, @@ -38,8 +38,16 @@ import org.apache.spark.util.Utils * So, when adding new fields, take into consideration that the whole object can be serialized for * shipping off at any time to consumers of the SparkListener interface. */ -@DeveloperApi -class TaskMetrics(val hostname: String) extends Serializable { +private[spark] class TaskMetrics( + internalMetricsToAccums: Map[String, Accumulator[Long]], + val hostname: String = TaskMetrics.getCachedHostName) + extends Serializable { + + import InternalAccumulator._ + + def this(host: String) { + this(TaskMetrics.newAccumMap, host) + } // Needed for Java def this() { @@ -47,19 +55,27 @@ class TaskMetrics(val hostname: String) extends Serializable { } /** - * Time taken on the executor to deserialize this task + * Return the internal accumulator associated with the specified metric, assuming it exists. */ - private var _executorDeserializeTime: Long = _ - def executorDeserializeTime: Long = _executorDeserializeTime - private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value + private def getAccum(metricName: String): Accumulator[Long] = { + assert(internalMetricsToAccums.contains(metricName), s"metric $metricName is missing") + internalMetricsToAccums(metricName) + } + + /** + * Time taken on the executor to deserialize this task. + */ + private val _executorDeserializeTime: Accumulator[Long] = getAccum(EXECUTOR_DESERIALIZE_TIME) + def executorDeserializeTime: Long = _executorDeserializeTime.value + def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) /** * Time the executor spends actually running the task (including fetching shuffle data) */ - private var _executorRunTime: Long = _ - def executorRunTime: Long = _executorRunTime - private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value + private val _executorRunTime: Accumulator[Long] = getAccum(EXECUTOR_RUN_TIME) + def executorRunTime: Long = _executorRunTime.value + def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) /** * The number of bytes this task transmitted back to the driver as the TaskResult @@ -242,6 +258,13 @@ private[spark] object TaskMetrics { val canonicalHost = hostNameCache.putIfAbsent(host, host) if (canonicalHost != null) canonicalHost else host } + + /** + * Construct a set of new accumulators indexed by metric name. + */ + private def newAccumMap: Map[String, Accumulator[Long]] = { + InternalAccumulator.create().map { accum => (accum.name.get, accum) }.toMap + } } /** diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 5b84acf40be4..6eae7b721196 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -168,8 +168,8 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val collectedAccums = taskContext.collectAccumulators() assert(internalMetricsToAccums.size > 0) assert(internalMetricsToAccums.values.forall(_.isInternal)) - assert(internalMetricsToAccums.contains(TEST_ACCUMULATOR)) - val testAccum = internalMetricsToAccums(TEST_ACCUMULATOR) + assert(internalMetricsToAccums.contains(TEST_ACCUM)) + val testAccum = internalMetricsToAccums(TEST_ACCUM) assert(collectedInternalAccums.size === internalMetricsToAccums.size) assert(collectedInternalAccums.size === collectedAccums.size) assert(collectedInternalAccums.contains(testAccum.id)) @@ -183,7 +183,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 + TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -193,11 +193,11 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(stageInfos.size === 1) assert(taskInfos.size === numPartitions) // The accumulator values should be merged in the stage - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUM) assert(stageAccum.value.toLong === numPartitions) // The accumulator should be updated locally on each task val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) + val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUM) assert(taskAccum.update.isDefined) assert(taskAccum.update.get.toLong === 1) taskAccum.value.toLong @@ -218,17 +218,17 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 + TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 10 + TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 100 + TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -237,9 +237,9 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val stageInfos = listener.getCompletedStageInfos assert(stageInfos.size === 3) val (firstStageAccum, secondStageAccum, thirdStageAccum) = - (findAccumulableInfo(stageInfos(0).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(1).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(2).accumulables.values, TEST_ACCUMULATOR)) + (findAccumulableInfo(stageInfos(0).accumulables.values, TEST_ACCUM), + findAccumulableInfo(stageInfos(1).accumulables.values, TEST_ACCUM), + findAccumulableInfo(stageInfos(2).accumulables.values, TEST_ACCUM)) assert(firstStageAccum.value.toLong === numPartitions) assert(secondStageAccum.value.toLong === numPartitions * 10) assert(thirdStageAccum.value.toLong === numPartitions * 2 * 100) @@ -278,7 +278,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => val taskContext = TaskContext.get() - taskContext.internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 + taskContext.internalMetricsToAccumulators(TEST_ACCUM) += 1 // Fail the first attempts of a subset of the tasks if (failCondition(i) && taskContext.attemptNumber() == 0) { throw new Exception("Failing a task intentionally.") @@ -291,13 +291,13 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val taskInfos = listener.getCompletedTaskInfos assert(stageInfos.size === 1) assert(taskInfos.size === numPartitions + numFailedPartitions) - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUM) // We should not double count values in the merged accumulator assert(stageAccum.value.toLong === numPartitions) val taskAccumValues = taskInfos.flatMap { taskInfo => if (!taskInfo.failed) { // If a task succeeded, its update value should always be 1 - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) + val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUM) assert(taskAccum.update.isDefined) assert(taskAccum.update.get.toLong === 1) Some(taskAccum.value.toLong) From a4ca6b2de82dac5f255cd40ace83c77b508ac43d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 4 Jan 2016 13:54:05 -0800 Subject: [PATCH 03/65] Migrate a few more easy metrics --- .../scala/org/apache/spark/Accumulators.scala | 18 +++++--- .../apache/spark/executor/TaskMetrics.scala | 44 +++++++++---------- 2 files changed, 33 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 4e30b39b1b36..164c226c4005 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -96,7 +96,7 @@ class Accumulable[R, T] private[spark] ( * Normally, a user will not want to use this version, but will instead call `+=`. * @param term the other `R` that will get merged with this */ - def ++= (term: R) { value_ = param.addInPlace(value_, term)} + def ++= (term: R) { value_ = param.addInPlace(value_, term) } /** * Merge two accumulable objects together @@ -104,7 +104,7 @@ class Accumulable[R, T] private[spark] ( * Normally, a user will not want to use this version, but will instead call `add`. * @param term the other `R` that will get merged with this */ - def merge(term: R) { value_ = param.addInPlace(value_, term)} + def merge(term: R) { value_ = param.addInPlace(value_, term) } /** * Access the accumulator's current value; only allowed on master. @@ -139,10 +139,6 @@ class Accumulable[R, T] private[spark] ( // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() - // TODO: fix me? - // value_ = zero - // deserialized = true - // Automatically register the accumulator when it is deserialized with the task closure. // // Note internal accumulators sent with task are deserialized before the TaskContext is created @@ -354,6 +350,11 @@ private[spark] object Accumulators extends Logging { private[spark] object InternalAccumulator { val EXECUTOR_DESERIALIZE_TIME = "executorDeserializeTime" val EXECUTOR_RUN_TIME = "executorRunTime" + val RESULT_SIZE = "resultSize" + val JVM_GC_TIME = "jvmGCTime" + val RESULT_SERIALIZATION_TIME = "resultSerializationTime" + val MEMORY_BYTES_SPILLED = "memoryBytesSpilled" + val DISK_BYTES_SPILLED = "diskBytesSpilled" val PEAK_EXECUTION_MEMORY = "peakExecutionMemory" val TEST_ACCUM = "testAccumulator" @@ -373,6 +374,11 @@ private[spark] object InternalAccumulator { Seq( newMetric(EXECUTOR_DESERIALIZE_TIME), newMetric(EXECUTOR_RUN_TIME), + newMetric(RESULT_SIZE), + newMetric(JVM_GC_TIME), + newMetric(RESULT_SERIALIZATION_TIME), + newMetric(MEMORY_BYTES_SPILLED), + newMetric(DISK_BYTES_SPILLED), // Execution memory refers to the memory used by internal data structures created // during shuffles, aggregations and joins. The value of this accumulator should be // approximately the sum of the peak sizes across all such data structures created diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 639d5336699c..fdee70f3d6fd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -69,49 +69,47 @@ private[spark] class TaskMetrics( def executorDeserializeTime: Long = _executorDeserializeTime.value def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) - /** - * Time the executor spends actually running the task (including fetching shuffle data) + * Time the executor spends actually running the task (including fetching shuffle data). */ private val _executorRunTime: Accumulator[Long] = getAccum(EXECUTOR_RUN_TIME) def executorRunTime: Long = _executorRunTime.value def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) /** - * The number of bytes this task transmitted back to the driver as the TaskResult + * The number of bytes this task transmitted back to the driver as the TaskResult. */ - private var _resultSize: Long = _ - def resultSize: Long = _resultSize - private[spark] def setResultSize(value: Long) = _resultSize = value - + private val _resultSize: Accumulator[Long] = getAccum(RESULT_SIZE) + def resultSize: Long = _resultSize.value + def setResultSize(v: Long) = _resultSize.setValue(v) /** - * Amount of time the JVM spent in garbage collection while executing this task + * Amount of time the JVM spent in garbage collection while executing this task. */ - private var _jvmGCTime: Long = _ - def jvmGCTime: Long = _jvmGCTime - private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value + private val _jvmGCTime: Accumulator[Long] = getAccum(JVM_GC_TIME) + def jvmGCTime: Long = _jvmGCTime.value + def setJvmGCTime(v: Long) = _jvmGCTime.setValue(v) /** - * Amount of time spent serializing the task result + * Amount of time spent serializing the task result. */ - private var _resultSerializationTime: Long = _ - def resultSerializationTime: Long = _resultSerializationTime - private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value + private val _resultSerializationTime: Accumulator[Long] = getAccum(RESULT_SERIALIZATION_TIME) + def resultSerializationTime: Long = _resultSerializationTime.value + def setResultSerializationTime(v: Long) = _resultSerializationTime.setValue(v) /** - * The number of in-memory bytes spilled by this task + * The number of in-memory bytes spilled by this task. */ - private var _memoryBytesSpilled: Long = _ - def memoryBytesSpilled: Long = _memoryBytesSpilled - private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value + private val _memoryBytesSpilled: Accumulator[Long] = getAccum(MEMORY_BYTES_SPILLED) + def memoryBytesSpilled: Long = _memoryBytesSpilled.value + def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) /** - * The number of on-disk bytes spilled by this task + * The number of on-disk bytes spilled by this task. */ - private var _diskBytesSpilled: Long = _ - def diskBytesSpilled: Long = _diskBytesSpilled - private[spark] def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value + private val _diskBytesSpilled: Accumulator[Long] = getAccum(DISK_BYTES_SPILLED) + def diskBytesSpilled: Long = _diskBytesSpilled.value + def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read From 373898e7bf7f887fc008638625072ed1b68d8359 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 4 Jan 2016 15:39:37 -0800 Subject: [PATCH 04/65] ShuffleReadMetrics + namespacing accumulators This commit ports ShuffleReadMetrics to use accumulators, preserving as much of the existing semantics as possible. It also introduces a nicer way to organize all the internal accumulators by namespacing them. --- .../shuffle/sort/UnsafeShuffleWriter.java | 9 +- .../scala/org/apache/spark/Accumulators.scala | 61 +++-- .../scala/org/apache/spark/Aggregator.scala | 3 +- .../apache/spark/executor/TaskMetrics.scala | 215 +++++++++++------- .../org/apache/spark/rdd/CoGroupedRDD.scala | 3 +- .../shuffle/BlockStoreShuffleReader.scala | 3 +- .../org/apache/spark/status/api/v1/api.scala | 6 +- .../org/apache/spark/ui/jobs/StagePage.scala | 15 +- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../util/collection/ExternalSorter.scala | 3 +- .../org/apache/spark/AccumulatorSuite.scala | 7 +- .../org/apache/spark/ui/StagePageSuite.scala | 4 +- .../org/apache/spark/sql/execution/Sort.scala | 10 +- .../TungstenAggregationIterator.scala | 6 +- .../execution/joins/BroadcastHashJoin.scala | 3 +- .../joins/BroadcastHashOuterJoin.scala | 3 +- .../joins/BroadcastLeftSemiJoinHash.scala | 3 +- .../spark/sql/execution/ReferenceSort.scala | 3 +- .../sql/util/DataFrameCallbackSuite.scala | 4 +- 19 files changed, 217 insertions(+), 156 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 744c3008ca50..d6001c47f452 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -445,13 +445,8 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th @Override public Option stop(boolean success) { try { - // Update task metrics from accumulators (null in UnsafeShuffleWriterSuite) - Map> internalAccumulators = - taskContext.internalMetricsToAccumulators(); - if (internalAccumulators != null) { - internalAccumulators.apply(InternalAccumulator.PEAK_EXECUTION_MEMORY()) - .add(getPeakMemoryUsedBytes()); - } + // TODO: this will throw NPE + taskContext.taskMetrics().incPeakExecutionMemory(getPeakMemoryUsedBytes()); if (stopping) { return Option.apply(null); diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 164c226c4005..df15e976e0f1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -348,6 +348,8 @@ private[spark] object Accumulators extends Logging { } private[spark] object InternalAccumulator { + + // Names of internal metrics val EXECUTOR_DESERIALIZE_TIME = "executorDeserializeTime" val EXECUTOR_RUN_TIME = "executorRunTime" val RESULT_SIZE = "resultSize" @@ -358,6 +360,21 @@ private[spark] object InternalAccumulator { val PEAK_EXECUTION_MEMORY = "peakExecutionMemory" val TEST_ACCUM = "testAccumulator" + // Names of shuffle read metrics + object shuffleRead { + val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" + val REMOTE_BYTES_READ = "remoteBytesRead" + val LOCAL_BYTES_READ = "localBytesRead" + val FETCH_WAIT_TIME = "fetchWaitTime" + val RECORDS_READ = "recordsRead" + } + + // Accumulator name prefixes + val METRICS_PREFIX = "metrics." + val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." + val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." + /** * Create a new internal Long accumulator with the specified name. */ @@ -367,24 +384,36 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking internal metrics. - * Note: this method does not register accumulators for clean up. + * Note: this method does not register accumulators for cleanup. */ def create(): Seq[Accumulator[Long]] = { - val maybeTestAccumulator = sys.props.get("spark.testing").map(_ => newMetric(TEST_ACCUM)).toSeq - Seq( - newMetric(EXECUTOR_DESERIALIZE_TIME), - newMetric(EXECUTOR_RUN_TIME), - newMetric(RESULT_SIZE), - newMetric(JVM_GC_TIME), - newMetric(RESULT_SERIALIZATION_TIME), - newMetric(MEMORY_BYTES_SPILLED), - newMetric(DISK_BYTES_SPILLED), - // Execution memory refers to the memory used by internal data structures created - // during shuffles, aggregations and joins. The value of this accumulator should be - // approximately the sum of the peak sizes across all such data structures created - // in this task. For SQL jobs, this only tracks all unsafe operators and ExternalSort. - newMetric(PEAK_EXECUTION_MEMORY) - ) ++ maybeTestAccumulator + val metricNames = Seq[String]( + EXECUTOR_DESERIALIZE_TIME, + EXECUTOR_RUN_TIME, + RESULT_SIZE, + JVM_GC_TIME, + RESULT_SERIALIZATION_TIME, + MEMORY_BYTES_SPILLED, + DISK_BYTES_SPILLED, + PEAK_EXECUTION_MEMORY) ++ + // For testing only + sys.props.get("spark.testing").map(_ => TEST_ACCUM).toSeq + metricNames.map { m => newMetric(METRICS_PREFIX + m) } ++ createShuffleReadMetrics() + } + + /** + * Accumulators for tracking shuffle read metrics. + * Note: this method does not register accumulators for cleanup. + */ + def createShuffleReadMetrics(): Seq[Accumulator[Long]] = { + val metricNames = Seq[String]( + shuffleRead.REMOTE_BLOCKS_FETCHED, + shuffleRead.LOCAL_BLOCKS_FETCHED, + shuffleRead.REMOTE_BYTES_READ, + shuffleRead.LOCAL_BYTES_READ, + shuffleRead.FETCH_WAIT_TIME, + shuffleRead.RECORDS_READ) + metricNames.map { m => newMetric(SHUFFLE_READ_METRICS_PREFIX + m) } } /** diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 7196e57d5d2e..240a035ca96d 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -65,8 +65,7 @@ case class Aggregator[K, V, C] ( Option(context).foreach { c => c.taskMetrics().incMemoryBytesSpilled(map.memoryBytesSpilled) c.taskMetrics().incDiskBytesSpilled(map.diskBytesSpilled) - c.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(map.peakMemoryUsedBytes) + c.taskMetrics().incPeakExecutionMemory(map.peakMemoryUsedBytes) } } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index fdee70f3d6fd..f045c5061452 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -39,7 +39,7 @@ import org.apache.spark.util.Utils * shipping off at any time to consumers of the SparkListener interface. */ private[spark] class TaskMetrics( - internalMetricsToAccums: Map[String, Accumulator[Long]], + accumMap: Map[String, Accumulator[Long]], val hostname: String = TaskMetrics.getCachedHostName) extends Serializable { @@ -57,9 +57,10 @@ private[spark] class TaskMetrics( /** * Return the internal accumulator associated with the specified metric, assuming it exists. */ - private def getAccum(metricName: String): Accumulator[Long] = { - assert(internalMetricsToAccums.contains(metricName), s"metric $metricName is missing") - internalMetricsToAccums(metricName) + private def getAccum(baseName: String): Accumulator[Long] = { + val fullName = METRICS_PREFIX + baseName + assert(accumMap.contains(fullName), s"metric '$fullName' is missing") + accumMap(fullName) } /** @@ -112,48 +113,97 @@ private[spark] class TaskMetrics( def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) /** - * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read - * are stored here. + * ... + * + * Execution memory refers to the memory used by internal data structures created + * during shuffles, aggregations and joins. The value of this accumulator should be + * approximately the sum of the peak sizes across all such data structures created + * in this task. For SQL jobs, this only tracks all unsafe operators and ExternalSort. */ - private var _inputMetrics: Option[InputMetrics] = None + private val _peakExecutionMemory: Accumulator[Long] = getAccum(PEAK_EXECUTION_MEMORY) + def peakExecutionMemory: Long = _peakExecutionMemory.value + def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) - def inputMetrics: Option[InputMetrics] = _inputMetrics + + /* ================================== * + | SHUFFLE READ METRICS | + * ================================== */ /** - * This should only be used when recreating TaskMetrics, not when updating input metrics in - * executors + * Aggregated [[ShuffleReadMetrics]] across all shuffle dependencies. */ - private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { - _inputMetrics = inputMetrics + private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics + def setShuffleReadMetrics(metrics: Option[ShuffleReadMetrics]): Unit = { + _shuffleReadMetrics = metrics } /** - * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much - * data was written are stored here. + * Temporary list of [[ShuffleReadMetrics]], one per shuffle dependency. + * + * A task may have multiple shuffle readers for multiple dependencies. To avoid synchronization + * issues from readers in different threads, in-progress tasks use a [[ShuffleReadMetrics]] for + * each dependency and merge these metrics before reporting them to the driver. */ - var outputMetrics: Option[OutputMetrics] = None + @transient private lazy val tempShuffleReadMetrics = new ArrayBuffer[ShuffleReadMetrics] /** - * If this task reads from shuffle output, metrics on getting shuffle data will be collected here. - * This includes read metrics aggregated over all the task's shuffle dependencies. + * Create a temporary [[ShuffleReadMetrics]] for a particular shuffle dependency. + * All values set in this [[ShuffleReadMetrics]] will be merged synchronously later. */ - private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { + val readMetrics = new ShuffleReadMetrics + tempShuffleReadMetrics += readMetrics + readMetrics + } + + /** + * Merge values across all temporary [[ShuffleReadMetrics]] into `_shuffleReadMetrics`. + */ + def updateShuffleReadMetrics(): Unit = synchronized { + val agg = _shuffleReadMetrics.getOrElse { + val metrics = new ShuffleReadMetrics(accumMap) + _shuffleReadMetrics = Some(metrics) + metrics + } + agg.setRemoteBlocksFetched(tempShuffleReadMetrics.map(_.remoteBlocksFetched).sum) + agg.setLocalBlocksFetched(tempShuffleReadMetrics.map(_.localBlocksFetched).sum) + agg.setFetchWaitTime(tempShuffleReadMetrics.map(_.fetchWaitTime).sum) + agg.setRemoteBytesRead(tempShuffleReadMetrics.map(_.remoteBytesRead).sum) + agg.setLocalBytesRead(tempShuffleReadMetrics.map(_.localBytesRead).sum) + agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) + } + + + + - def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics + + + + /** - * This should only be used when recreating TaskMetrics, not when updating read metrics in - * executors. + * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read + * are stored here. */ - private[spark] def setShuffleReadMetrics(shuffleReadMetrics: Option[ShuffleReadMetrics]) { - _shuffleReadMetrics = shuffleReadMetrics + private var _inputMetrics: Option[InputMetrics] = None + + def inputMetrics: Option[InputMetrics] = _inputMetrics + + /** + * This should only be used when recreating TaskMetrics, not when updating input metrics in + * executors + */ + private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { + _inputMetrics = inputMetrics } /** - * ShuffleReadMetrics per dependency for collecting independently while task is in progress. + * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much + * data was written are stored here. */ - @transient private lazy val depsShuffleReadMetrics: ArrayBuffer[ShuffleReadMetrics] = - new ArrayBuffer[ShuffleReadMetrics]() + var outputMetrics: Option[OutputMetrics] = None /** * If this task writes to shuffle output, metrics on the written shuffle data will be collected @@ -166,18 +216,6 @@ private[spark] class TaskMetrics( */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - /** - * A task may have multiple shuffle readers for multiple dependencies. To avoid synchronization - * issues from readers in different threads, in-progress tasks use a ShuffleReadMetrics for each - * dependency, and merge these metrics before reporting them to the driver. This method returns - * a ShuffleReadMetrics for a dependency and registers it for merging later. - */ - private [spark] def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { - val readMetrics = new ShuffleReadMetrics() - depsShuffleReadMetrics += readMetrics - readMetrics - } - /** * Returns the input metrics object that the task should use. Currently, if * there exists an input metric with the same readMethod, we return that one @@ -203,24 +241,6 @@ private[spark] class TaskMetrics( } } - /** - * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. - */ - private[spark] def updateShuffleReadMetrics(): Unit = synchronized { - if (!depsShuffleReadMetrics.isEmpty) { - val merged = new ShuffleReadMetrics() - for (depMetrics <- depsShuffleReadMetrics) { - merged.incFetchWaitTime(depMetrics.fetchWaitTime) - merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) - merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) - merged.incRemoteBytesRead(depMetrics.remoteBytesRead) - merged.incLocalBytesRead(depMetrics.localBytesRead) - merged.incRecordsRead(depMetrics.recordsRead) - } - _shuffleReadMetrics = Some(merged) - } - } - private[spark] def updateInputMetrics(): Unit = synchronized { inputMetrics.foreach(_.updateBytesRead()) } @@ -352,64 +372,89 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { } /** - * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. */ -@DeveloperApi -class ShuffleReadMetrics extends Serializable { +private[spark] class ShuffleReadMetrics(accumMap: Map[String, Accumulator[Long]]) + extends Serializable { + + import InternalAccumulator.shuffleRead._ + /** - * Number of remote blocks fetched in this shuffle by this task + * Construct a temporary [[ShuffleReadMetrics]], one for each shuffle dependency. */ - private var _remoteBlocksFetched: Int = _ - def remoteBlocksFetched: Int = _remoteBlocksFetched - private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value + def this() { + this(InternalAccumulator.createShuffleReadMetrics().map { a => (a.name.get, a) }.toMap) + } /** - * Number of local blocks fetched in this shuffle by this task + * Return the internal accumulator associated with the specified metric, assuming it exists. + * TODO: duplicate code alert! */ - private var _localBlocksFetched: Int = _ - def localBlocksFetched: Int = _localBlocksFetched - private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value + private def getAccum(baseName: String): Accumulator[Long] = { + val fullName = InternalAccumulator.SHUFFLE_READ_METRICS_PREFIX + baseName + assert(accumMap.contains(fullName), s"metric '$fullName' is missing") + accumMap(fullName) + } + + /** + * Number of remote blocks fetched in this shuffle by this task. + */ + private val _remoteBlocksFetched: Accumulator[Long] = getAccum(REMOTE_BLOCKS_FETCHED) + def remoteBlocksFetched: Long = _remoteBlocksFetched.value + def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) + def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) + + /** + * Number of local blocks fetched in this shuffle by this task. + */ + private val _localBlocksFetched: Accumulator[Long] = getAccum(LOCAL_BLOCKS_FETCHED) + def localBlocksFetched: Long = _localBlocksFetched.value + def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) + def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) /** * Time the task spent waiting for remote shuffle blocks. This only includes the time * blocking on shuffle input data. For instance if block B is being fetched while the task is * still not finished processing block A, it is not considered to be blocking on block B. */ - private var _fetchWaitTime: Long = _ - def fetchWaitTime: Long = _fetchWaitTime - private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value + private val _fetchWaitTime: Accumulator[Long] = getAccum(FETCH_WAIT_TIME) + def fetchWaitTime: Long = _fetchWaitTime.value + def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) + def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) /** - * Total number of remote bytes read from the shuffle by this task + * Total number of remote bytes read from the shuffle by this task. */ - private var _remoteBytesRead: Long = _ - def remoteBytesRead: Long = _remoteBytesRead - private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value + private val _remoteBytesRead: Accumulator[Long] = getAccum(REMOTE_BYTES_READ) + def remoteBytesRead: Long = _remoteBytesRead.value + def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) + def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ - private var _localBytesRead: Long = _ - def localBytesRead: Long = _localBytesRead - private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value + private val _localBytesRead: Accumulator[Long] = getAccum(LOCAL_BYTES_READ) + def localBytesRead: Long = _localBytesRead.value + def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) + def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) /** - * Total bytes fetched in the shuffle by this task (both remote and local). + * Total number of records read from the shuffle by this task. */ - def totalBytesRead: Long = _remoteBytesRead + _localBytesRead + private val _recordsRead: Accumulator[Long] = getAccum(RECORDS_READ) + def recordsRead: Long = _recordsRead.value + def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) /** - * Number of blocks fetched in this shuffle by this task (remote or local) + * Total bytes fetched in the shuffle by this task (both remote and local). */ - def totalBlocksFetched: Int = _remoteBlocksFetched + _localBlocksFetched + def totalBytesRead: Long = remoteBytesRead + localBytesRead /** - * Total number of records read from the shuffle by this task + * Number of blocks fetched in this shuffle by this task (remote or local). */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - private[spark] def incRecordsRead(value: Long) = _recordsRead += value + def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 3a0ca1d81329..ad3282e70a22 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -154,8 +154,7 @@ class CoGroupedRDD[K: ClassTag]( } context.taskMetrics().incMemoryBytesSpilled(map.memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(map.diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(map.peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(map.peakMemoryUsedBytes) new InterruptibleIterator(context, map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index b0abda4a81b8..31629630ca71 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -103,8 +103,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( sorter.insertAll(aggregatedIter) context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) case None => aggregatedIter diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5feb1dc2e5b7..28b5b173f02e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -178,11 +178,11 @@ class OutputMetrics private[spark]( val recordsWritten: Long) class ShuffleReadMetrics private[spark]( - val remoteBlocksFetched: Int, - val localBlocksFetched: Int, + val remoteBlocksFetched: Long, + val localBlocksFetched: Long, val fetchWaitTime: Long, val remoteBytesRead: Long, - val totalBlocksFetched: Int, + val totalBlocksFetched: Long, val recordsRead: Long) class ShuffleWriteMetrics private[spark]( 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 08e7576b0c08..4c9b4f867b7d 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 @@ -402,12 +402,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(gettingResultTimes) - val peakExecutionMemory = validTasks.map { case TaskUIData(info, _, _) => - info.accumulables - .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.update.getOrElse("0").toLong } - .getOrElse(0L) - .toDouble + val peakExecutionMemory = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.peakExecutionMemory.toDouble } val peakExecutionMemoryQuantiles = { @@ -889,15 +885,12 @@ private[ui] class TaskDataSource( val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val (taskInternalAccumulables, taskExternalAccumulables) = + val (_, taskExternalAccumulables) = info.accumulables.partition(_.internal) val externalAccumulableReadable = taskExternalAccumulables.map { acc => StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update.get}") } - val peakExecutionMemoryUsed = taskInternalAccumulables - .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.update.getOrElse("0").toLong } - .getOrElse(0L) + val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L) val maybeInput = metrics.flatMap(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead).getOrElse(0L) 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 4e8eaeeefda5..cadfef1939ac 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -744,12 +744,12 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics - metrics.incRemoteBlocksFetched((json \ "Remote Blocks Fetched").extract[Int]) - metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int]) - metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) - metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) - metrics.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) - metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) + metrics.setRemoteBlocksFetched((json \ "Remote Blocks Fetched").extract[Long]) + metrics.setLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Long]) + metrics.setFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) + metrics.setRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) + metrics.setLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) + metrics.setRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) metrics } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 44b1d90667e6..262827fe9dd5 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -680,8 +680,7 @@ private[spark] class ExternalSorter[K, V, C]( context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) lengths } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 6eae7b721196..1b895beecf44 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -316,9 +316,12 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex private[spark] object AccumulatorSuite { + import InternalAccumulator._ + /** * Run one or more Spark jobs and verify that the peak execution memory accumulator * is updated afterwards. + * TODO: assert it's also set in task metrics? */ def verifyPeakExecutionMemorySet( sc: SparkContext, @@ -330,12 +333,12 @@ private[spark] object AccumulatorSuite { if (jobId == 0) { // The first job is a dummy one to verify that the accumulator does not already exist val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) - assert(!accums.exists(_.name == InternalAccumulator.PEAK_EXECUTION_MEMORY)) + assert(!accums.exists(_.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY)) } else { // In the subsequent jobs, verify that peak execution memory is updated val accum = listener.getCompletedStageInfos .flatMap(_.accumulables.values) - .find(_.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) + .find(_.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY) .getOrElse { throw new TestFailedException( s"peak execution memory accumulator not set in '$testName'", 0) 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 86699e7f5695..ed29e2aa0850 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -31,6 +31,8 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener class StagePageSuite extends SparkFunSuite with LocalSparkContext { + import InternalAccumulator._ + test("peak execution memory only displayed if unsafe is enabled") { val unsafeConf = "spark.sql.unsafe.enabled" val conf = new SparkConf(false).set(unsafeConf, "true") @@ -80,7 +82,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) val peakExecutionMemory = 10 - taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY, + taskInfo.accumulables += new AccumulableInfo(0, METRICS_PREFIX + PEAK_EXECUTION_MEMORY, Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala index 73dc8cb98447..1c88baa8cd02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala @@ -79,17 +79,17 @@ case class Sort( sorter.setTestSpillFrequency(testSpillFrequency) } + val metrics = TaskContext.get().taskMetrics() + // Remember spill data size of this task before execute this operator so that we can // figure out how many bytes we spilled for this operator. - val spillSizeBefore = TaskContext.get().taskMetrics().memoryBytesSpilled - + val spillSizeBefore = metrics.memoryBytesSpilled val sortedIterator = sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]]) dataSize += sorter.getPeakMemoryUsage - spillSize += TaskContext.get().taskMetrics().memoryBytesSpilled - spillSizeBefore + spillSize += metrics.memoryBytesSpilled - spillSizeBefore + metrics.incPeakExecutionMemory(sorter.getPeakMemoryUsage) - TaskContext.get().internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.getPeakMemoryUsage) sortedIterator } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 582fdbe54706..6069a51d3934 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -418,10 +418,10 @@ class TungstenAggregationIterator( val mapMemory = hashMap.getPeakMemoryUsedBytes val sorterMemory = Option(externalSorter).map(_.getPeakMemoryUsedBytes).getOrElse(0L) val peakMemory = Math.max(mapMemory, sorterMemory) + val metrics = TaskContext.get().taskMetrics() dataSize += peakMemory - spillSize += TaskContext.get().taskMetrics().memoryBytesSpilled - spillSizeBefore - TaskContext.get().internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(peakMemory) + spillSize += metrics.memoryBytesSpilled - spillSizeBefore + metrics.incPeakExecutionMemory(peakMemory) } numOutputRows += 1 res diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 1d381e2eaef3..fc313a08841d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -110,8 +110,7 @@ case class BroadcastHashJoin( val hashedRelation = broadcastRelation.value hashedRelation match { case unsafe: UnsafeHashedRelation => - TaskContext.get().internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(unsafe.getUnsafeSize) + TaskContext.get().taskMetrics().incPeakExecutionMemory(unsafe.getUnsafeSize) case _ => } hashJoin(streamedIter, numStreamedRows, hashedRelation, numOutputRows) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index ab81bd7b3fc0..7840d1601e57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -119,8 +119,7 @@ case class BroadcastHashOuterJoin( hashTable match { case unsafe: UnsafeHashedRelation => - TaskContext.get().internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(unsafe.getUnsafeSize) + TaskContext.get().taskMetrics().incPeakExecutionMemory(unsafe.getUnsafeSize) case _ => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 004407b2e692..8929dc3af191 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -66,8 +66,7 @@ case class BroadcastLeftSemiJoinHash( val hashedRelation = broadcastedRelation.value hashedRelation match { case unsafe: UnsafeHashedRelation => - TaskContext.get().internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(unsafe.getUnsafeSize) + TaskContext.get().taskMetrics().incPeakExecutionMemory(unsafe.getUnsafeSize) case _ => } hashSemiJoin(streamIter, numLeftRows, hashedRelation, numOutputRows) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index 9575d26fd123..273937fa8ce9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -49,8 +49,7 @@ case class ReferenceSort( val context = TaskContext.get() context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) CompletionIterator[InternalRow, Iterator[InternalRow]](baseIterator, sorter.stop()) }, preservesPartitioning = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index b46b0d2f6040..b12492a18a35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -116,6 +116,8 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { // A easy fix is to create a new SQLMetric(including new MetricValue, MetricParam, etc.), but we // can do it later because the impact is just too small (1048576 tasks for 1 MB). ignore("get size metrics by callback") { + import InternalAccumulator._ + val metrics = ArrayBuffer.empty[Long] val listener = new QueryExecutionListener { // Only test successful case here, so no need to implement `onFailure` @@ -137,7 +139,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { def getPeakExecutionMemory(stageId: Int): Long = { val peakMemoryAccumulator = sparkListener.getCompletedStageInfos(stageId).accumulables - .filter(_._2.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) + .filter(_._2.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY) assert(peakMemoryAccumulator.size == 1) peakMemoryAccumulator.head._2.value.toLong From e74632c3e0ed6b20f52f112a0cfdad27e036a827 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 13:01:08 -0800 Subject: [PATCH 05/65] General code cleanup --- .../scala/org/apache/spark/Accumulators.scala | 82 ++++--- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 215 +++++++++++------- .../shuffle/BlockStoreShuffleReader.scala | 4 +- .../storage/ShuffleBlockFetcherIterator.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 95 ++++---- .../org/apache/spark/AccumulatorSuite.scala | 4 +- .../spark/executor/TaskMetricsSuite.scala | 2 +- .../org/apache/spark/ui/StagePageSuite.scala | 2 +- .../ui/jobs/JobProgressListenerSuite.scala | 8 +- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- .../sql/util/DataFrameCallbackSuite.scala | 2 +- 12 files changed, 237 insertions(+), 185 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index df15e976e0f1..6b44392b0d83 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -37,6 +37,8 @@ import org.apache.spark.util.Utils * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are * accumulating a set. You will add items to the set, and you will union two sets together. * + * TODO: document thread-safety. + * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` * @param name human-readable name for use in Spark's web UI @@ -82,13 +84,13 @@ class Accumulable[R, T] private[spark] ( * Add more data to this accumulator / accumulable * @param term the data to add */ - def += (term: T) { value_ = param.addAccumulator(value_, term) } + def += (term: T): Unit = { value_ = param.addAccumulator(value_, term) } /** * Add more data to this accumulator / accumulable * @param term the data to add */ - def add(term: T) { value_ = param.addAccumulator(value_, term) } + def add(term: T): Unit = { value_ = param.addAccumulator(value_, term) } /** * Merge two accumulable objects together @@ -96,7 +98,7 @@ class Accumulable[R, T] private[spark] ( * Normally, a user will not want to use this version, but will instead call `+=`. * @param term the other `R` that will get merged with this */ - def ++= (term: R) { value_ = param.addInPlace(value_, term) } + def ++= (term: R): Unit = { value_ = param.addInPlace(value_, term) } /** * Merge two accumulable objects together @@ -104,7 +106,7 @@ class Accumulable[R, T] private[spark] ( * Normally, a user will not want to use this version, but will instead call `add`. * @param term the other `R` that will get merged with this */ - def merge(term: R) { value_ = param.addInPlace(value_, term) } + def merge(term: R): Unit = { value_ = param.addInPlace(value_, term) } /** * Access the accumulator's current value; only allowed on master. @@ -119,20 +121,22 @@ class Accumulable[R, T] private[spark] ( * * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. + * + * TODO: probably don't need this. */ def localValue: R = value_ /** - * Set the accumulator's value; only allowed on master. + * Set the accumulator's value. */ - def value_= (newValue: R) { + def value_= (newValue: R): Unit = { value_ = newValue } /** - * Set the accumulator's value; only allowed on master + * Set the accumulator's value. */ - def setValue(newValue: R) { + def setValue(newValue: R): Unit = { this.value = newValue } @@ -349,31 +353,33 @@ private[spark] object Accumulators extends Logging { private[spark] object InternalAccumulator { - // Names of internal metrics - val EXECUTOR_DESERIALIZE_TIME = "executorDeserializeTime" - val EXECUTOR_RUN_TIME = "executorRunTime" - val RESULT_SIZE = "resultSize" - val JVM_GC_TIME = "jvmGCTime" - val RESULT_SERIALIZATION_TIME = "resultSerializationTime" - val MEMORY_BYTES_SPILLED = "memoryBytesSpilled" - val DISK_BYTES_SPILLED = "diskBytesSpilled" - val PEAK_EXECUTION_MEMORY = "peakExecutionMemory" - val TEST_ACCUM = "testAccumulator" + // Names of internal task level metrics + val EXECUTOR_DESERIALIZE_TIME = "metrics.executorDeserializeTime" + val EXECUTOR_RUN_TIME = "metrics.executorRunTime" + val RESULT_SIZE = "metrics.resultSize" + val JVM_GC_TIME = "metrics.jvmGCTime" + val RESULT_SERIALIZATION_TIME = "metrics.resultSerializationTime" + val MEMORY_BYTES_SPILLED = "metrics.memoryBytesSpilled" + val DISK_BYTES_SPILLED = "metrics.diskBytesSpilled" + val PEAK_EXECUTION_MEMORY = "metrics.peakExecutionMemory" + val TEST_ACCUM = "metrics.testAccumulator" // Names of shuffle read metrics object shuffleRead { - val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" - val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" - val REMOTE_BYTES_READ = "remoteBytesRead" - val LOCAL_BYTES_READ = "localBytesRead" - val FETCH_WAIT_TIME = "fetchWaitTime" - val RECORDS_READ = "recordsRead" + val REMOTE_BLOCKS_FETCHED = "metrics.shuffle.read.remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = "metrics.shuffle.read.localBlocksFetched" + val REMOTE_BYTES_READ = "metrics.shuffle.read.remoteBytesRead" + val LOCAL_BYTES_READ = "metrics.shuffle.read.localBytesRead" + val FETCH_WAIT_TIME = "metrics.shuffle.read.fetchWaitTime" + val RECORDS_READ = "metrics.shuffle.read.recordsRead" } - // Accumulator name prefixes - val METRICS_PREFIX = "metrics." - val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." - val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." + // Names of shuffle write metrics + object shuffleWrite { + val SHUFFLE_BYTES_WRITTEN = "metrics.shuffle.write.shuffleBytesWritten" + val SHUFFLE_RECORDS_WRITTEN = "metrics.shuffle.write.shuffleRecordsWritten" + val SHUFFLE_WRITE_TIME = "metrics.shuffle.write.shuffleWriteTime" + } /** * Create a new internal Long accumulator with the specified name. @@ -398,22 +404,32 @@ private[spark] object InternalAccumulator { PEAK_EXECUTION_MEMORY) ++ // For testing only sys.props.get("spark.testing").map(_ => TEST_ACCUM).toSeq - metricNames.map { m => newMetric(METRICS_PREFIX + m) } ++ createShuffleReadMetrics() + metricNames.map(newMetric) ++ createShuffleReadAccums() } /** * Accumulators for tracking shuffle read metrics. * Note: this method does not register accumulators for cleanup. */ - def createShuffleReadMetrics(): Seq[Accumulator[Long]] = { - val metricNames = Seq[String]( + def createShuffleReadAccums(): Seq[Accumulator[Long]] = { + Seq[String]( shuffleRead.REMOTE_BLOCKS_FETCHED, shuffleRead.LOCAL_BLOCKS_FETCHED, shuffleRead.REMOTE_BYTES_READ, shuffleRead.LOCAL_BYTES_READ, shuffleRead.FETCH_WAIT_TIME, - shuffleRead.RECORDS_READ) - metricNames.map { m => newMetric(SHUFFLE_READ_METRICS_PREFIX + m) } + shuffleRead.RECORDS_READ).map(newMetric) + } + + /** + * Accumulators for tracking shuffle read metrics. + * Note: this method does not register accumulators for cleanup. + */ + def createShuffleWriteAccums(): Seq[Accumulator[Long]] = { + Seq[String]( + shuffleWrite.SHUFFLE_BYTES_WRITTEN, + shuffleWrite.SHUFFLE_RECORDS_WRITTEN, + shuffleWrite.SHUFFLE_WRITE_TIME).map(newMetric) } /** diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 9b1418436424..75d7e34d60eb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -425,7 +425,7 @@ private[spark] class Executor( for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { taskRunner.task.metrics.foreach { metrics => - metrics.updateShuffleReadMetrics() + metrics.mergeShuffleReadMetrics() metrics.updateInputMetrics() metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) metrics.updateAccumulators() diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index f045c5061452..2b99b5266c7c 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -27,6 +27,7 @@ import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils + /** * Metrics tracked during the execution of a task. * @@ -38,7 +39,8 @@ import org.apache.spark.util.Utils * So, when adding new fields, take into consideration that the whole object can be serialized for * shipping off at any time to consumers of the SparkListener interface. */ -private[spark] class TaskMetrics( +@deprecated("TaskMetrics will be made private in a future version.", "2.0.0") +class TaskMetrics( accumMap: Map[String, Accumulator[Long]], val hostname: String = TaskMetrics.getCachedHostName) extends Serializable { @@ -46,7 +48,7 @@ private[spark] class TaskMetrics( import InternalAccumulator._ def this(host: String) { - this(TaskMetrics.newAccumMap, host) + this(InternalAccumulator.create().map { accum => (accum.name.get, accum) }.toMap, host) } // Needed for Java @@ -54,89 +56,87 @@ private[spark] class TaskMetrics( this(TaskMetrics.getCachedHostName) } + // Each metric is internally represented as an accumulator + private val _executorDeserializeTime: Accumulator[Long] = getAccum(EXECUTOR_DESERIALIZE_TIME) + private val _executorRunTime: Accumulator[Long] = getAccum(EXECUTOR_RUN_TIME) + private val _resultSize: Accumulator[Long] = getAccum(RESULT_SIZE) + private val _jvmGCTime: Accumulator[Long] = getAccum(JVM_GC_TIME) + private val _resultSerializationTime: Accumulator[Long] = getAccum(RESULT_SERIALIZATION_TIME) + private val _memoryBytesSpilled: Accumulator[Long] = getAccum(MEMORY_BYTES_SPILLED) + private val _diskBytesSpilled: Accumulator[Long] = getAccum(DISK_BYTES_SPILLED) + private val _peakExecutionMemory: Accumulator[Long] = getAccum(PEAK_EXECUTION_MEMORY) + /** * Return the internal accumulator associated with the specified metric, assuming it exists. */ - private def getAccum(baseName: String): Accumulator[Long] = { - val fullName = METRICS_PREFIX + baseName - assert(accumMap.contains(fullName), s"metric '$fullName' is missing") - accumMap(fullName) + private def getAccum(name: String): Accumulator[Long] = { + assert(accumMap.contains(name), s"metric '$name' is missing") + accumMap(name) } /** * Time taken on the executor to deserialize this task. */ - private val _executorDeserializeTime: Accumulator[Long] = getAccum(EXECUTOR_DESERIALIZE_TIME) def executorDeserializeTime: Long = _executorDeserializeTime.value - def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) /** * Time the executor spends actually running the task (including fetching shuffle data). */ - private val _executorRunTime: Accumulator[Long] = getAccum(EXECUTOR_RUN_TIME) def executorRunTime: Long = _executorRunTime.value - def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) /** * The number of bytes this task transmitted back to the driver as the TaskResult. */ - private val _resultSize: Accumulator[Long] = getAccum(RESULT_SIZE) def resultSize: Long = _resultSize.value - def setResultSize(v: Long) = _resultSize.setValue(v) /** * Amount of time the JVM spent in garbage collection while executing this task. */ - private val _jvmGCTime: Accumulator[Long] = getAccum(JVM_GC_TIME) def jvmGCTime: Long = _jvmGCTime.value - def setJvmGCTime(v: Long) = _jvmGCTime.setValue(v) /** * Amount of time spent serializing the task result. */ - private val _resultSerializationTime: Accumulator[Long] = getAccum(RESULT_SERIALIZATION_TIME) def resultSerializationTime: Long = _resultSerializationTime.value - def setResultSerializationTime(v: Long) = _resultSerializationTime.setValue(v) /** * The number of in-memory bytes spilled by this task. */ - private val _memoryBytesSpilled: Accumulator[Long] = getAccum(MEMORY_BYTES_SPILLED) def memoryBytesSpilled: Long = _memoryBytesSpilled.value - def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) /** * The number of on-disk bytes spilled by this task. */ - private val _diskBytesSpilled: Accumulator[Long] = getAccum(DISK_BYTES_SPILLED) def diskBytesSpilled: Long = _diskBytesSpilled.value - def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) /** - * ... - * - * Execution memory refers to the memory used by internal data structures created - * during shuffles, aggregations and joins. The value of this accumulator should be - * approximately the sum of the peak sizes across all such data structures created - * in this task. For SQL jobs, this only tracks all unsafe operators and ExternalSort. + * Peak memory used by internal data structures created during shuffles, aggregations and + * joins. The value of this accumulator should be approximately the sum of the peak sizes + * across all such data structures created in this task. For SQL jobs, this only tracks all + * unsafe operators and ExternalSort. */ - private val _peakExecutionMemory: Accumulator[Long] = getAccum(PEAK_EXECUTION_MEMORY) def peakExecutionMemory: Long = _peakExecutionMemory.value - def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + + private[spark] def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) + private[spark] def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) + private[spark] def setResultSize(v: Long) = _resultSize.setValue(v) + private[spark] def setJvmGCTime(v: Long) = _jvmGCTime.setValue(v) + private[spark] def setResultSerializationTime(v: Long) = _resultSerializationTime.setValue(v) + private[spark] def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) + private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) + private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) /* ================================== * | SHUFFLE READ METRICS | * ================================== */ + private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + /** * Aggregated [[ShuffleReadMetrics]] across all shuffle dependencies. */ - private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics - def setShuffleReadMetrics(metrics: Option[ShuffleReadMetrics]): Unit = { - _shuffleReadMetrics = metrics - } /** * Temporary list of [[ShuffleReadMetrics]], one per shuffle dependency. @@ -149,18 +149,22 @@ private[spark] class TaskMetrics( /** * Create a temporary [[ShuffleReadMetrics]] for a particular shuffle dependency. - * All values set in this [[ShuffleReadMetrics]] will be merged synchronously later. + * + * All usages are expected to be followed by a call to [[mergeShuffleReadMetrics]], + * which merges the temporary values synchronously. */ - def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { - val readMetrics = new ShuffleReadMetrics + private[spark] def registerTempShuffleReadMetrics(): ShuffleReadMetrics = synchronized { + val tempAccumMap = createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap + val readMetrics = new ShuffleReadMetrics(tempAccumMap) tempShuffleReadMetrics += readMetrics readMetrics } /** * Merge values across all temporary [[ShuffleReadMetrics]] into `_shuffleReadMetrics`. + * This is expected to be called on executor heartbeat and at the end of a task. */ - def updateShuffleReadMetrics(): Unit = synchronized { + private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { val agg = _shuffleReadMetrics.getOrElse { val metrics = new ShuffleReadMetrics(accumMap) _shuffleReadMetrics = Some(metrics) @@ -181,7 +185,16 @@ private[spark] class TaskMetrics( - + + + + + + + + /* ================================== * + | OTHER THINGS... WIP | + * ================================== */ /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read @@ -262,6 +275,7 @@ private[spark] class TaskMetrics( } } + private[spark] object TaskMetrics { private val hostNameCache = new ConcurrentHashMap[String, String]() @@ -276,15 +290,9 @@ private[spark] object TaskMetrics { val canonicalHost = hostNameCache.putIfAbsent(host, host) if (canonicalHost != null) canonicalHost else host } - - /** - * Construct a set of new accumulators indexed by metric name. - */ - private def newAccumMap: Map[String, Accumulator[Long]] = { - InternalAccumulator.create().map { accum => (accum.name.get, accum) }.toMap - } } + /** * :: DeveloperApi :: * Method by which input data was read. Network means that the data was read over the network @@ -296,6 +304,7 @@ object DataReadMethod extends Enumeration with Serializable { val Memory, Disk, Hadoop, Network = Value } + /** * :: DeveloperApi :: * Method by which output data was written. @@ -306,6 +315,7 @@ object DataWriteMethod extends Enumeration with Serializable { val Hadoop = Value } + /** * :: DeveloperApi :: * Metrics about reading input data. @@ -350,6 +360,7 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { } } + /** * :: DeveloperApi :: * Metrics about writing output data. @@ -371,80 +382,61 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value } + /** * Metrics pertaining to shuffle data read in a given task. */ -private[spark] class ShuffleReadMetrics(accumMap: Map[String, Accumulator[Long]]) +@deprecated("ShuffleReadMetrics will be made private in a future version.", "2.0.0") +class ShuffleReadMetrics private ( + _remoteBlocksFetched: Accumulator[Long], + _localBlocksFetched: Accumulator[Long], + _remoteBytesRead: Accumulator[Long], + _localBytesRead: Accumulator[Long], + _fetchWaitTime: Accumulator[Long], + _recordsRead: Accumulator[Long]) extends Serializable { - import InternalAccumulator.shuffleRead._ - - /** - * Construct a temporary [[ShuffleReadMetrics]], one for each shuffle dependency. - */ - def this() { - this(InternalAccumulator.createShuffleReadMetrics().map { a => (a.name.get, a) }.toMap) - } - - /** - * Return the internal accumulator associated with the specified metric, assuming it exists. - * TODO: duplicate code alert! - */ - private def getAccum(baseName: String): Accumulator[Long] = { - val fullName = InternalAccumulator.SHUFFLE_READ_METRICS_PREFIX + baseName - assert(accumMap.contains(fullName), s"metric '$fullName' is missing") - accumMap(fullName) + private[executor] def this(accumMap: Map[String, Accumulator[Long]]) { + this( + accumMap(InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), + accumMap(InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), + accumMap(InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), + accumMap(InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), + accumMap(InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), + accumMap(InternalAccumulator.shuffleRead.RECORDS_READ)) } /** * Number of remote blocks fetched in this shuffle by this task. */ - private val _remoteBlocksFetched: Accumulator[Long] = getAccum(REMOTE_BLOCKS_FETCHED) def remoteBlocksFetched: Long = _remoteBlocksFetched.value - def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) - def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) /** * Number of local blocks fetched in this shuffle by this task. */ - private val _localBlocksFetched: Accumulator[Long] = getAccum(LOCAL_BLOCKS_FETCHED) def localBlocksFetched: Long = _localBlocksFetched.value - def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) - def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) - - /** - * Time the task spent waiting for remote shuffle blocks. This only includes the time - * blocking on shuffle input data. For instance if block B is being fetched while the task is - * still not finished processing block A, it is not considered to be blocking on block B. - */ - private val _fetchWaitTime: Accumulator[Long] = getAccum(FETCH_WAIT_TIME) - def fetchWaitTime: Long = _fetchWaitTime.value - def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) - def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) /** * Total number of remote bytes read from the shuffle by this task. */ - private val _remoteBytesRead: Accumulator[Long] = getAccum(REMOTE_BYTES_READ) def remoteBytesRead: Long = _remoteBytesRead.value - def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) - def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ - private val _localBytesRead: Accumulator[Long] = getAccum(LOCAL_BYTES_READ) def localBytesRead: Long = _localBytesRead.value - def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) - def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) + + /** + * Time the task spent waiting for remote shuffle blocks. This only includes the time + * blocking on shuffle input data. For instance if block B is being fetched while the task is + * still not finished processing block A, it is not considered to be blocking on block B. + */ + def fetchWaitTime: Long = _fetchWaitTime.value /** * Total number of records read from the shuffle by this task. */ - private val _recordsRead: Accumulator[Long] = getAccum(RECORDS_READ) def recordsRead: Long = _recordsRead.value - def incRecordsRead(v: Long): Unit = _recordsRead.add(v) - def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) /** * Total bytes fetched in the shuffle by this task (both remote and local). @@ -455,11 +447,27 @@ private[spark] class ShuffleReadMetrics(accumMap: Map[String, Accumulator[Long]] * Number of blocks fetched in this shuffle by this task (remote or local). */ def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched + + private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) + private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) + private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) + private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) + private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + + private[spark] def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) + private[spark] def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) + private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) + private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) + private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) + private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) } + /** * :: DeveloperApi :: * Metrics pertaining to shuffle data written in a given task. + * TODO: REMOVE ME. */ @DeveloperApi class ShuffleWriteMetrics extends Serializable { @@ -486,3 +494,36 @@ class ShuffleWriteMetrics extends Serializable { private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value } + + +///** +// * Metrics pertaining to shuffle data written in a given task. +// */ +//@deprecated("ShuffleWriteMetrics will be made private in a future version.", "2.0.0") +//class ShuffleWriteMetrics private[executor] ( +// _shuffleBytesWritten: Accumulator[Long], +// _shuffleRecordsWritten: Accumulator[Long], +// _shuffleWriteTime: Accumulator[Long]) +// extends Serializable { +// +// /** +// * Number of bytes written for the shuffle by this task. +// */ +// def shuffleBytesWritten: Long = _shuffleBytesWritten.value +// +// /** +// * Total number of records written to the shuffle by this task. +// */ +// def shuffleRecordsWritten: Long = _shuffleRecordsWritten.value +// +// /** +// * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. +// */ +// def shuffleWriteTime: Long = _shuffleWriteTime.value +// +// private[spark] def incShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.add(v) +// private[spark] def incShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.add(v) +// private[spark] def incShuffleWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) +// private[spark] def decShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.sub(v) +// private[spark] def decShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.sub(v) +//} diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 31629630ca71..acbe16001f5b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -65,13 +65,13 @@ private[spark] class BlockStoreShuffleReader[K, C]( } // Update the context task metrics for each record read. - val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + val readMetrics = context.taskMetrics.registerTempShuffleReadMetrics() val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( recordIter.map(record => { readMetrics.incRecordsRead(1) record }), - context.taskMetrics().updateShuffleReadMetrics()) + context.taskMetrics().mergeShuffleReadMetrics()) // An interruptible iterator must be used here in order to support task cancellation val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 0d0448feb5b0..d6deca32ab4a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -101,7 +101,7 @@ final class ShuffleBlockFetcherIterator( /** Current bytes in flight from our requests */ private[this] var bytesInFlight = 0L - private[this] val shuffleMetrics = context.taskMetrics().createShuffleReadMetricsForDependency() + private[this] val shuffleMetrics = context.taskMetrics().registerTempShuffleReadMetrics() /** * Whether the iterator is still active. If isZombie is true, the callback interface will no 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 cadfef1939ac..baaf6659dfa7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -292,14 +292,33 @@ private[spark] object JsonProtocol { } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { - val shuffleReadMetrics = - taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) - val shuffleWriteMetrics = - taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val inputMetrics = - taskMetrics.inputMetrics.map(inputMetricsToJson).getOrElse(JNothing) + val shuffleReadMetrics: JValue = + taskMetrics.shuffleReadMetrics.map { rm => + ("Remote Blocks Fetched" -> rm.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> rm.localBlocksFetched) ~ + ("Fetch Wait Time" -> rm.fetchWaitTime) ~ + ("Remote Bytes Read" -> rm.remoteBytesRead) ~ + ("Local Bytes Read" -> rm.localBytesRead) ~ + ("Total Records Read" -> rm.recordsRead) + }.getOrElse(JNothing) + val shuffleWriteMetrics: JValue = + taskMetrics.shuffleWriteMetrics.map { wm => + ("Shuffle Bytes Written" -> wm.shuffleBytesWritten) ~ + ("Shuffle Write Time" -> wm.shuffleWriteTime) ~ + ("Shuffle Records Written" -> wm.shuffleRecordsWritten) + }.getOrElse(JNothing) + val inputMetrics: JValue = + taskMetrics.inputMetrics.map { im => + ("Data Read Method" -> im.readMethod.toString) ~ + ("Bytes Read" -> im.bytesRead) ~ + ("Records Read" -> im.recordsRead) + }.getOrElse(JNothing) val outputMetrics = - taskMetrics.outputMetrics.map(outputMetricsToJson).getOrElse(JNothing) + taskMetrics.outputMetrics.map { om => + ("Data Write Method" -> om.writeMethod.toString) ~ + ("Bytes Written" -> om.bytesWritten) ~ + ("Records Written" -> om.recordsWritten) + }.getOrElse(JNothing) val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => @@ -322,33 +341,6 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } - def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { - ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ - ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ - ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ - ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ - ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~ - ("Total Records Read" -> shuffleReadMetrics.recordsRead) - } - - def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { - ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) ~ - ("Shuffle Records Written" -> shuffleWriteMetrics.shuffleRecordsWritten) - } - - def inputMetricsToJson(inputMetrics: InputMetrics): JValue = { - ("Data Read Method" -> inputMetrics.readMethod.toString) ~ - ("Bytes Read" -> inputMetrics.bytesRead) ~ - ("Records Read" -> inputMetrics.recordsRead) - } - - def outputMetricsToJson(outputMetrics: OutputMetrics): JValue = { - ("Data Write Method" -> outputMetrics.writeMethod.toString) ~ - ("Bytes Written" -> outputMetrics.bytesWritten) ~ - ("Records Written" -> outputMetrics.recordsWritten) - } - def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { @@ -723,8 +715,19 @@ private[spark] object JsonProtocol { metrics.setResultSerializationTime((json \ "Result Serialization Time").extract[Long]) metrics.incMemoryBytesSpilled((json \ "Memory Bytes Spilled").extract[Long]) metrics.incDiskBytesSpilled((json \ "Disk Bytes Spilled").extract[Long]) - metrics.setShuffleReadMetrics( - Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson)) + + // Shuffle read metrics + Utils.jsonOption(json \ "Shuffle Read Metrics").foreach { readJson => + val readMetrics = metrics.registerTempShuffleReadMetrics() + readMetrics.setRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Long]) + readMetrics.setLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Long]) + readMetrics.setRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) + readMetrics.setLocalBytesRead((readJson \ "Local Bytes Read").extractOpt[Long].getOrElse(0L)) + readMetrics.setFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) + readMetrics.setRecordsRead((readJson \ "Total Records Read").extractOpt[Long].getOrElse(0L)) + metrics.mergeShuffleReadMetrics() + } + metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) metrics.setInputMetrics( @@ -742,18 +745,8 @@ private[spark] object JsonProtocol { metrics } - def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { - val metrics = new ShuffleReadMetrics - metrics.setRemoteBlocksFetched((json \ "Remote Blocks Fetched").extract[Long]) - metrics.setLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Long]) - metrics.setFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) - metrics.setRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) - metrics.setLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) - metrics.setRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) - metrics - } - - def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { + // TODO: kill this method + private def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { val metrics = new ShuffleWriteMetrics metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) @@ -762,7 +755,8 @@ private[spark] object JsonProtocol { metrics } - def inputMetricsFromJson(json: JValue): InputMetrics = { + // TODO: kill this method + private def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( DataReadMethod.withName((json \ "Data Read Method").extract[String])) metrics.incBytesRead((json \ "Bytes Read").extract[Long]) @@ -770,7 +764,8 @@ private[spark] object JsonProtocol { metrics } - def outputMetricsFromJson(json: JValue): OutputMetrics = { + // TODO: kill this method + private def outputMetricsFromJson(json: JValue): OutputMetrics = { val metrics = new OutputMetrics( DataWriteMethod.withName((json \ "Data Write Method").extract[String])) metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 1b895beecf44..eac3fe9fbb4e 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -333,12 +333,12 @@ private[spark] object AccumulatorSuite { if (jobId == 0) { // The first job is a dummy one to verify that the accumulator does not already exist val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) - assert(!accums.exists(_.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY)) + assert(!accums.exists(_.name == PEAK_EXECUTION_MEMORY)) } else { // In the subsequent jobs, verify that peak execution memory is updated val accum = listener.getCompletedStageInfos .flatMap(_.accumulables.values) - .find(_.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY) + .find(_.name == PEAK_EXECUTION_MEMORY) .getOrElse { throw new TestFailedException( s"peak execution memory accumulator not set in '$testName'", 0) diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 8275fd87764c..e5ec2aa1be35 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.SparkFunSuite class TaskMetricsSuite extends SparkFunSuite { test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") { val taskMetrics = new TaskMetrics() - taskMetrics.updateShuffleReadMetrics() + taskMetrics.mergeShuffleReadMetrics() assert(taskMetrics.shuffleReadMetrics.isEmpty) } } 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 ed29e2aa0850..bc8a4299d9c3 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -82,7 +82,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) val peakExecutionMemory = 10 - taskInfo.accumulables += new AccumulableInfo(0, METRICS_PREFIX + PEAK_EXECUTION_MEMORY, + taskInfo.accumulables += new AccumulableInfo(0, PEAK_EXECUTION_MEMORY, Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index e02f5a1b20fe..bb9d05bff763 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -184,12 +184,12 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val conf = new SparkConf() val listener = new JobProgressListener(conf) val taskMetrics = new TaskMetrics() - val shuffleReadMetrics = new ShuffleReadMetrics() + val shuffleReadMetrics = taskMetrics.registerTempShuffleReadMetrics() assert(listener.stageIdToData.size === 0) // finish this task, should get updated shuffleRead shuffleReadMetrics.incRemoteBytesRead(1000) - taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) + taskMetrics.mergeShuffleReadMetrics() var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 var task = new ShuffleMapTask(0) @@ -270,13 +270,13 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with def makeTaskMetrics(base: Int): TaskMetrics = { val taskMetrics = new TaskMetrics() - val shuffleReadMetrics = new ShuffleReadMetrics() + val shuffleReadMetrics = taskMetrics.registerTempShuffleReadMetrics() val shuffleWriteMetrics = new ShuffleWriteMetrics() - taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) shuffleReadMetrics.incRemoteBytesRead(base + 1) shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) + taskMetrics.mergeShuffleReadMetrics() shuffleWriteMetrics.incShuffleBytesWritten(base + 3) taskMetrics.setExecutorRunTime(base + 4) taskMetrics.incDiskBytesSpilled(base + 5) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a8f2ce03e90b..719e902c6bde 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -778,14 +778,14 @@ class JsonProtocolSuite extends SparkFunSuite { inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1) t.setInputMetrics(Some(inputMetrics)) } else { - val sr = new ShuffleReadMetrics + val sr = t.registerTempShuffleReadMetrics() sr.incRemoteBytesRead(b + d) sr.incLocalBlocksFetched(e) sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1) sr.incLocalBytesRead(a + f) - t.setShuffleReadMetrics(Some(sr)) + t.mergeShuffleReadMetrics() } if (hasOutput) { val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index b12492a18a35..d1cfa9bf6736 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -139,7 +139,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { def getPeakExecutionMemory(stageId: Int): Long = { val peakMemoryAccumulator = sparkListener.getCompletedStageInfos(stageId).accumulables - .filter(_._2.name == METRICS_PREFIX + PEAK_EXECUTION_MEMORY) + .filter(_._2.name == PEAK_EXECUTION_MEMORY) assert(peakMemoryAccumulator.size == 1) peakMemoryAccumulator.head._2.value.toLong From 7e74bf38ac13e6eb148b15dd95e4241de1f7c118 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 13:47:24 -0800 Subject: [PATCH 06/65] ShuffleWriteMetrics --- .../sort/BypassMergeSortShuffleWriter.java | 3 +- .../shuffle/sort/ShuffleExternalSorter.java | 3 +- .../shuffle/sort/UnsafeShuffleWriter.java | 3 +- .../spark/unsafe/map/BytesToBytesMap.java | 3 +- .../unsafe/sort/UnsafeExternalSorter.java | 4 +- .../scala/org/apache/spark/Accumulators.scala | 5 +- .../apache/spark/executor/TaskMetrics.scala | 143 +++++++++++------- .../shuffle/hash/HashShuffleWriter.scala | 4 +- .../shuffle/sort/SortShuffleWriter.scala | 3 - .../org/apache/spark/util/JsonProtocol.scala | 21 ++- .../collection/ExternalAppendOnlyMap.scala | 4 +- .../util/collection/ExternalSorter.scala | 12 +- .../storage/DiskBlockObjectWriterSuite.scala | 6 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../execution/UnsafeRowSerializerSuite.scala | 1 - .../spark/tools/StoragePerfTester.scala | 2 +- 16 files changed, 119 insertions(+), 100 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a1a1fb01426a..a7aae092bc22 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -114,8 +114,7 @@ public BypassMergeSortShuffleWriter( this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); - this.writeMetrics = new ShuffleWriteMetrics(); - taskContext.taskMetrics().shuffleWriteMetrics_$eq(Option.apply(writeMetrics)); + this.writeMetrics = taskContext.taskMetrics().registerShuffleWriteMetrics(); this.serializer = Serializer.getSerializer(dep.serializer()); this.shuffleBlockResolver = shuffleBlockResolver; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 9affff80143d..223b91854ef6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -31,6 +31,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.executor.ShuffleWriteMetrics$; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.DummySerializerInstance; @@ -138,7 +139,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // We're spilling, so bytes written should be counted towards spill rather than write. // Create a dummy WriteMetrics object to absorb these metrics, since we don't want to count // them towards shuffle bytes written. - writeMetricsToUse = new ShuffleWriteMetrics(); + writeMetricsToUse = ShuffleWriteMetrics$.MODULE$.createDummy(); } // This call performs the actual sort. diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index d6001c47f452..b1c8c227d315 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -119,8 +119,7 @@ public UnsafeShuffleWriter( this.shuffleId = dep.shuffleId(); this.serializer = Serializer.getSerializer(dep.serializer()).newInstance(); this.partitioner = dep.partitioner(); - this.writeMetrics = new ShuffleWriteMetrics(); - taskContext.taskMetrics().shuffleWriteMetrics_$eq(Option.apply(writeMetrics)); + this.writeMetrics = taskContext.taskMetrics().registerShuffleWriteMetrics(); this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 3387f9a4177c..665893227d21 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -30,6 +30,7 @@ import org.apache.spark.SparkEnv; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.executor.ShuffleWriteMetrics$; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.storage.BlockManager; @@ -340,7 +341,7 @@ public long spill(long numBytes) throws IOException { } // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); + ShuffleWriteMetrics writeMetrics = ShuffleWriteMetrics$.MODULE$.createDummy(); long released = 0L; while (dataPages.size() > 0) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 79d74b23ceae..6334e17fcd2e 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -120,9 +120,7 @@ private UnsafeExternalSorter( // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; this.fileBufferSizeBytes = 32 * 1024; - // TODO: metrics tracking + integration with shuffle write metrics - // need to connect the write metrics to task metrics so we count the spill IO somewhere. - this.writeMetrics = new ShuffleWriteMetrics(); + this.writeMetrics = taskContext.taskMetrics().registerShuffleWriteMetrics(); if (existingInMemorySorter == null) { this.inMemSorter = new UnsafeInMemorySorter( diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6b44392b0d83..aa3d33fb50a4 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -402,9 +402,10 @@ private[spark] object InternalAccumulator { MEMORY_BYTES_SPILLED, DISK_BYTES_SPILLED, PEAK_EXECUTION_MEMORY) ++ - // For testing only sys.props.get("spark.testing").map(_ => TEST_ACCUM).toSeq - metricNames.map(newMetric) ++ createShuffleReadAccums() + metricNames.map(newMetric) ++ + createShuffleReadAccums() ++ + createShuffleWriteAccums() } /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 2b99b5266c7c..e68b46d562a1 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -127,6 +127,28 @@ class TaskMetrics( private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + /* =================================== * + | SHUFFLE WRITE METRICS | + * =================================== */ + + private var _shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + /** + * Metrics related to shuffle write, defined only in shuffle map stages. + */ + def shuffleWriteMetrics: Option[ShuffleWriteMetrics] = _shuffleWriteMetrics + + /** + * Get or create a new [[ShuffleWriteMetrics]] associated with this task. + */ + def registerShuffleWriteMetrics(): ShuffleWriteMetrics = { + _shuffleWriteMetrics.getOrElse { + val metrics = new ShuffleWriteMetrics(accumMap) + _shuffleWriteMetrics = Some(metrics) + metrics + } + } + /* ================================== * | SHUFFLE READ METRICS | * ================================== */ @@ -134,7 +156,8 @@ class TaskMetrics( private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * Aggregated [[ShuffleReadMetrics]] across all shuffle dependencies. + * Metrics related to shuffle read aggregated across all shuffle dependencies. + * This is defined only if there are shuffle dependencies in this task. */ def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics @@ -154,8 +177,7 @@ class TaskMetrics( * which merges the temporary values synchronously. */ private[spark] def registerTempShuffleReadMetrics(): ShuffleReadMetrics = synchronized { - val tempAccumMap = createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap - val readMetrics = new ShuffleReadMetrics(tempAccumMap) + val readMetrics = ShuffleReadMetrics.createDummy() tempShuffleReadMetrics += readMetrics readMetrics } @@ -218,12 +240,6 @@ class TaskMetrics( */ var outputMetrics: Option[OutputMetrics] = None - /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected - * here - */ - var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None - /** * Storage statuses of any blocks that have been updated as a result of this task. */ @@ -463,67 +479,78 @@ class ShuffleReadMetrics private ( private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) } +private[spark] object ShuffleReadMetrics { + + /** + * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + */ + def createDummy(): ShuffleReadMetrics = { + new ShuffleReadMetrics( + InternalAccumulator.createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap) + } +} + /** - * :: DeveloperApi :: * Metrics pertaining to shuffle data written in a given task. - * TODO: REMOVE ME. */ -@DeveloperApi -class ShuffleWriteMetrics extends Serializable { +@deprecated("ShuffleWriteMetrics will be made private in a future version.", "2.0.0") +class ShuffleWriteMetrics private ( + _shuffleBytesWritten: Accumulator[Long], + _shuffleRecordsWritten: Accumulator[Long], + _shuffleWriteTime: Accumulator[Long]) + extends Serializable { + + private[executor] def this(accumMap: Map[String, Accumulator[Long]]) { + this( + accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_BYTES_WRITTEN), + accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_RECORDS_WRITTEN), + accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_WRITE_TIME)) + } + /** - * Number of bytes written for the shuffle by this task + * Number of bytes written for the shuffle by this task. */ - @volatile private var _shuffleBytesWritten: Long = _ - def shuffleBytesWritten: Long = _shuffleBytesWritten - private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value - private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value + def shuffleBytesWritten: Long = _shuffleBytesWritten.value /** - * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds + * Total number of records written to the shuffle by this task. */ - @volatile private var _shuffleWriteTime: Long = _ - def shuffleWriteTime: Long = _shuffleWriteTime - private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value + def shuffleRecordsWritten: Long = _shuffleRecordsWritten.value /** - * Total number of records written to the shuffle by this task + * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. */ - @volatile private var _shuffleRecordsWritten: Long = _ - def shuffleRecordsWritten: Long = _shuffleRecordsWritten - private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value - private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value + def shuffleWriteTime: Long = _shuffleWriteTime.value + + // TODO: these are not thread-safe. Is that OK? + + private[spark] def incShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.add(v) + private[spark] def incShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.add(v) + private[spark] def incShuffleWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) + private[spark] def decShuffleBytesWritten(v: Long): Unit = { + _shuffleBytesWritten.setValue(shuffleBytesWritten - v) + } + private[spark] def decShuffleRecordsWritten(v: Long): Unit = { + _shuffleRecordsWritten.setValue(shuffleRecordsWritten - v) + } } +private[spark] object ShuffleWriteMetrics { -///** -// * Metrics pertaining to shuffle data written in a given task. -// */ -//@deprecated("ShuffleWriteMetrics will be made private in a future version.", "2.0.0") -//class ShuffleWriteMetrics private[executor] ( -// _shuffleBytesWritten: Accumulator[Long], -// _shuffleRecordsWritten: Accumulator[Long], -// _shuffleWriteTime: Accumulator[Long]) -// extends Serializable { -// -// /** -// * Number of bytes written for the shuffle by this task. -// */ -// def shuffleBytesWritten: Long = _shuffleBytesWritten.value -// -// /** -// * Total number of records written to the shuffle by this task. -// */ -// def shuffleRecordsWritten: Long = _shuffleRecordsWritten.value -// -// /** -// * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. -// */ -// def shuffleWriteTime: Long = _shuffleWriteTime.value -// -// private[spark] def incShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.add(v) -// private[spark] def incShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.add(v) -// private[spark] def incShuffleWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) -// private[spark] def decShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.sub(v) -// private[spark] def decShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.sub(v) -//} + /** + * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + */ + def createDummy(): ShuffleWriteMetrics = { + new ShuffleWriteMetrics( + InternalAccumulator.createShuffleWriteAccums().map { acc => (acc.name.get, acc) }.toMap) + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 412bf70000da..35e0f2c50555 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -42,9 +42,7 @@ private[spark] class HashShuffleWriter[K, V]( // we don't try deleting files, etc twice. private var stopping = false - private val writeMetrics = new ShuffleWriteMetrics() - metrics.shuffleWriteMetrics = Some(writeMetrics) - + private val writeMetrics = metrics.registerShuffleWriteMetrics() private val blockManager = SparkEnv.get.blockManager private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index f83cf8859e58..896021a7b8b2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -45,9 +45,6 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null - private val writeMetrics = new ShuffleWriteMetrics() - context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) - /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { sorter = if (dep.mapSideCombine) { 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 baaf6659dfa7..fb2b70210b66 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -728,8 +728,15 @@ private[spark] object JsonProtocol { metrics.mergeShuffleReadMetrics() } - metrics.shuffleWriteMetrics = - Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + // Shuffle write metrics + Utils.jsonOption(json \ "Shuffle Write Metrics").foreach { writeJson => + val writeMetrics = metrics.registerShuffleWriteMetrics() + writeMetrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) + writeMetrics.incShuffleRecordsWritten((json \ "Shuffle Records Written") + .extractOpt[Long].getOrElse(0)) + writeMetrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) + } + metrics.setInputMetrics( Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) metrics.outputMetrics = @@ -745,16 +752,6 @@ private[spark] object JsonProtocol { metrics } - // TODO: kill this method - private def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { - val metrics = new ShuffleWriteMetrics - metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) - metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) - metrics.incShuffleRecordsWritten((json \ "Shuffle Records Written") - .extractOpt[Long].getOrElse(0)) - metrics - } - // TODO: kill this method private def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( 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 f6d81ee5bf05..7cb9444de784 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 @@ -181,7 +181,7 @@ class ExternalAppendOnlyMap[K, V, C]( */ override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempLocalBlock() - curWriteMetrics = new ShuffleWriteMetrics() + curWriteMetrics = ShuffleWriteMetrics.createDummy() var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 @@ -208,7 +208,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - curWriteMetrics = new ShuffleWriteMetrics() + curWriteMetrics = ShuffleWriteMetrics.createDummy() writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 262827fe9dd5..cfd4866c0e36 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -245,7 +245,7 @@ private[spark] class ExternalSorter[K, V, C]( var writer: DiskBlockObjectWriter = null def openWriter(): Unit = { assert (writer == null && spillMetrics == null) - spillMetrics = new ShuffleWriteMetrics + spillMetrics = ShuffleWriteMetrics.createDummy() writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics) } openWriter() @@ -644,6 +644,8 @@ private[spark] class ExternalSorter[K, V, C]( blockId: BlockId, outputFile: File): Array[Long] = { + val writeMetrics = context.taskMetrics().registerShuffleWriteMetrics() + // Track location of each range in the output file val lengths = new Array[Long](numPartitions) @@ -652,8 +654,8 @@ private[spark] class ExternalSorter[K, V, C]( val collection = if (aggregator.isDefined) map else buffer val it = collection.destructiveSortedWritablePartitionedIterator(comparator) while (it.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, - context.taskMetrics.shuffleWriteMetrics.get) + val writer = blockManager.getDiskWriter( + blockId, outputFile, serInstance, fileBufferSize, writeMetrics) val partitionId = it.nextPartition() while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(writer) @@ -666,8 +668,8 @@ private[spark] class ExternalSorter[K, V, C]( // We must perform merge-sort; get an iterator by partition and write everything directly. for ((id, elements) <- this.partitionedIterator) { if (elements.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, - context.taskMetrics.shuffleWriteMetrics.get) + val writer = blockManager.getDiskWriter( + blockId, outputFile, serInstance, fileBufferSize, writeMetrics) for (elem <- elements) { writer.write(elem._1, elem._2) } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 5d36617cfc44..bd0978ff259b 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -44,7 +44,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("verify write metrics") { val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() + val writeMetrics = ShuffleWriteMetrics.createDummy() val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) @@ -118,7 +118,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("commitAndClose() should be idempotent") { val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() + val writeMetrics = ShuffleWriteMetrics.createDummy() val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) for (i <- 1 to 1000) { @@ -136,7 +136,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("revertPartialWritesAndClose() should be idempotent") { val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() + val writeMetrics = ShuffleWriteMetrics.createDummy() val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) for (i <- 1 to 1000) { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index bb9d05bff763..421feadfc1f3 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -271,7 +271,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with def makeTaskMetrics(base: Int): TaskMetrics = { val taskMetrics = new TaskMetrics() val shuffleReadMetrics = taskMetrics.registerTempShuffleReadMetrics() - val shuffleWriteMetrics = new ShuffleWriteMetrics() + val shuffleWriteMetrics = taskMetrics.registerShuffleWriteMetrics() taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) shuffleReadMetrics.incRemoteBytesRead(base + 1) shuffleReadMetrics.incLocalBytesRead(base + 9) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 09e258299de5..3d0ccd7313d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -128,7 +128,6 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { assert(sorter.numSpills > 0) // Merging spilled files should not throw assertion error - taskContext.taskMetrics.shuffleWriteMetrics = Some(new ShuffleWriteMetrics) sorter.writePartitionedFile(ShuffleBlockId(0, 0, 0), outputFile) } { // Clean up diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8a5c7c0e730e..b0cbf6f11155 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -61,7 +61,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) + new KryoSerializer(sc.conf), ShuffleWriteMetrics.createDummy()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeKey, writeValue) From 396088d1fff77dd17d280ba3297c4b18d142b99f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 14:17:51 -0800 Subject: [PATCH 07/65] OutputMetrics --- .../scala/org/apache/spark/Accumulators.scala | 21 ++++++- .../apache/spark/executor/TaskMetrics.scala | 61 +++++++++++++------ .../apache/spark/rdd/PairRDDFunctions.scala | 23 +++---- .../org/apache/spark/util/JsonProtocol.scala | 28 ++++----- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 11 ++-- 6 files changed, 86 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index aa3d33fb50a4..ea31f92d5330 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -381,6 +381,12 @@ private[spark] object InternalAccumulator { val SHUFFLE_WRITE_TIME = "metrics.shuffle.write.shuffleWriteTime" } + // Names of output metrics + object output { + val BYTES_WRITTEN = "metrics.output.bytesWritten" + val RECORDS_WRITTEN = "metrics.output.recordsWritten" + } + /** * Create a new internal Long accumulator with the specified name. */ @@ -405,7 +411,8 @@ private[spark] object InternalAccumulator { sys.props.get("spark.testing").map(_ => TEST_ACCUM).toSeq metricNames.map(newMetric) ++ createShuffleReadAccums() ++ - createShuffleWriteAccums() + createShuffleWriteAccums() ++ + createOutputAccums() } /** @@ -423,7 +430,7 @@ private[spark] object InternalAccumulator { } /** - * Accumulators for tracking shuffle read metrics. + * Accumulators for tracking shuffle write metrics. * Note: this method does not register accumulators for cleanup. */ def createShuffleWriteAccums(): Seq[Accumulator[Long]] = { @@ -433,6 +440,16 @@ private[spark] object InternalAccumulator { shuffleWrite.SHUFFLE_WRITE_TIME).map(newMetric) } + /** + * Accumulators for tracking output metrics. + * Note: this method does not register accumulators for cleanup. + */ + private def createOutputAccums(): Seq[Accumulator[Long]] = { + Seq[String]( + output.BYTES_WRITTEN, + output.RECORDS_WRITTEN).map(newMetric) + } + /** * Accumulators for tracking internal metrics. * diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e68b46d562a1..fe12f0bc2743 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -200,11 +200,28 @@ class TaskMetrics( agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) } + /* ============================ * + | OUTPUT METRICS | + * ============================ */ + private var _outputMetrics: Option[OutputMetrics] = None + /** + * Metrics related to writing data externally (e.g. to a distributed filesystem), + * defined only tasks with output. + */ + def outputMetrics: Option[OutputMetrics] = _outputMetrics - - + /** + * Get or create a new [[OutputMetrics]] associated with this task. + */ + def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = { + _outputMetrics.getOrElse { + val metrics = new OutputMetrics(writeMethod, accumMap) + _outputMetrics = Some(metrics) + metrics + } + } @@ -234,12 +251,6 @@ class TaskMetrics( _inputMetrics = inputMetrics } - /** - * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much - * data was written are stored here. - */ - var outputMetrics: Option[OutputMetrics] = None - /** * Storage statuses of any blocks that have been updated as a result of this task. */ @@ -378,24 +389,36 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { /** - * :: DeveloperApi :: * Metrics about writing output data. */ -@DeveloperApi -case class OutputMetrics(writeMethod: DataWriteMethod.Value) { +@deprecated("OutputMetrics will be made private in a future version.", "2.0.0") +class OutputMetrics private ( + val writeMethod: DataWriteMethod.Value, + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long]) + extends Serializable { + + private[executor] def this( + writeMethod: DataWriteMethod.Value, + accumMap: Map[String, Accumulator[Long]]) { + this( + writeMethod, + accumMap(InternalAccumulator.output.BYTES_WRITTEN), + accumMap(InternalAccumulator.output.RECORDS_WRITTEN)) + } + /** - * Total bytes written + * Total number of bytes written. */ - private var _bytesWritten: Long = _ - def bytesWritten: Long = _bytesWritten - private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value + def bytesWritten: Long = _bytesWritten.value /** - * Total records written + * Total number of records written. */ - private var _recordsWritten: Long = 0L - def recordsWritten: Long = _recordsWritten - private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value + def recordsWritten: Long = _recordsWritten.value + + private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) + private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index b87230142532..9b32e2a1f03c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1098,8 +1098,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) - val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) - + val bytesWrittenCallback = SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback() + val outputMetrics = context.taskMetrics().registerOutputMetrics(DataWriteMethod.Hadoop) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K, V]] require(writer != null, "Unable to obtain RecordWriter") var recordsWritten = 0L @@ -1183,8 +1183,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // around by taking a mod. We expect that no task will be attempted 2 billion times. val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt - val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) - + val bytesWrittenCallback = SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback() + val outputMetrics = context.taskMetrics().registerOutputMetrics(DataWriteMethod.Hadoop) writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() var recordsWritten = 0L @@ -1210,17 +1210,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.commitJob() } - private def initHadoopOutputMetrics(context: TaskContext): (OutputMetrics, Option[() => Long]) = { - val bytesWrittenCallback = SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback() - val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) - if (bytesWrittenCallback.isDefined) { - context.taskMetrics.outputMetrics = Some(outputMetrics) - } - (outputMetrics, bytesWrittenCallback) - } - - private def maybeUpdateOutputMetrics(bytesWrittenCallback: Option[() => Long], - outputMetrics: OutputMetrics, recordsWritten: Long): Unit = { + private def maybeUpdateOutputMetrics( + bytesWrittenCallback: Option[() => Long], + outputMetrics: OutputMetrics, + recordsWritten: Long): Unit = { if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) { bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } outputMetrics.setRecordsWritten(recordsWritten) 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 fb2b70210b66..1ec797fa7983 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -724,23 +724,28 @@ private[spark] object JsonProtocol { readMetrics.setRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) readMetrics.setLocalBytesRead((readJson \ "Local Bytes Read").extractOpt[Long].getOrElse(0L)) readMetrics.setFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) - readMetrics.setRecordsRead((readJson \ "Total Records Read").extractOpt[Long].getOrElse(0L)) + readMetrics.setRecordsRead((readJson \ "Total Records Read").extract[Long]) metrics.mergeShuffleReadMetrics() } // Shuffle write metrics Utils.jsonOption(json \ "Shuffle Write Metrics").foreach { writeJson => val writeMetrics = metrics.registerShuffleWriteMetrics() - writeMetrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) - writeMetrics.incShuffleRecordsWritten((json \ "Shuffle Records Written") - .extractOpt[Long].getOrElse(0)) - writeMetrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) + writeMetrics.incShuffleBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) + writeMetrics.incShuffleRecordsWritten((writeJson \ "Shuffle Records Written").extract[Long]) + writeMetrics.incShuffleWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + } + + // Output metrics + Utils.jsonOption(json \ "Output Metrics").foreach { outJson => + val writeMethod = DataWriteMethod.withName((outJson \ "Data Write Method").extract[String]) + val outputMetrics = metrics.registerOutputMetrics(writeMethod) + outputMetrics.setBytesWritten((outJson \ "Bytes Written").extract[Long]) + outputMetrics.setRecordsWritten((outJson \ "Records Written").extract[Long]) } metrics.setInputMetrics( Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) - metrics.outputMetrics = - Utils.jsonOption(json \ "Output Metrics").map(outputMetricsFromJson) metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => value.extract[List[JValue]].map { block => @@ -761,15 +766,6 @@ private[spark] object JsonProtocol { metrics } - // TODO: kill this method - private def outputMetricsFromJson(json: JValue): OutputMetrics = { - val metrics = new OutputMetrics( - DataWriteMethod.withName((json \ "Data Write Method").extract[String])) - metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) - metrics.setRecordsWritten((json \ "Records Written").extractOpt[Long].getOrElse(0)) - metrics - } - def taskEndReasonFromJson(json: JValue): TaskEndReason = { val success = Utils.getFormattedClassName(Success) val resubmitted = Utils.getFormattedClassName(Resubmitted) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 421feadfc1f3..15af7dd22026 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -284,7 +284,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) taskMetrics.setInputMetrics(Some(inputMetrics)) inputMetrics.incBytesRead(base + 7) - val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) + val outputMetrics = taskMetrics.registerOutputMetrics(DataWriteMethod.Hadoop) taskMetrics.outputMetrics = Some(outputMetrics) outputMetrics.setBytesWritten(base + 8) taskMetrics diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 719e902c6bde..e4deeece4e35 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -274,14 +274,13 @@ class JsonProtocolSuite extends SparkFunSuite { assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } - test("ShuffleReadMetrics: Local bytes read and time taken backwards compatibility") { - // Metrics about local shuffle bytes read and local read time were added in 1.3.1. + test("ShuffleReadMetrics: Local bytes read backwards compatibility") { + // Metrics about local shuffle bytes read were added in 1.3.1. val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = false, hasOutput = false, hasRecords = false) assert(metrics.shuffleReadMetrics.nonEmpty) val newJson = JsonProtocol.taskMetricsToJson(metrics) val oldJson = newJson.removeField { case (field, _) => field == "Local Bytes Read" } - .removeField { case (field, _) => field == "Local Read Time" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.shuffleReadMetrics.get.localBytesRead == 0) } @@ -788,16 +787,14 @@ class JsonProtocolSuite extends SparkFunSuite { t.mergeShuffleReadMetrics() } if (hasOutput) { - val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) + val outputMetrics = t.registerOutputMetrics(DataWriteMethod.Hadoop) outputMetrics.setBytesWritten(a + b + c) outputMetrics.setRecordsWritten(if (hasRecords) (a + b + c)/100 else -1) - t.outputMetrics = Some(outputMetrics) } else { - val sw = new ShuffleWriteMetrics + val sw = t.registerShuffleWriteMetrics() sw.incShuffleBytesWritten(a + b + c) sw.incShuffleWriteTime(b + c + d) sw.incShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) - t.shuffleWriteMetrics = Some(sw) } // Make at most 6 blocks t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => From 0404e3e8f291f6cb8b007fb09d252ee2bb513308 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 15:52:31 -0800 Subject: [PATCH 08/65] InputMetrics This commit was a little tricky because it ripped the bytes read callback from TaskMetrics and related classes. It does change behavior in the sense that now we periodically update the number of bytes read (every 1000 records) instead of doing it every time we send an executor heartbeat. The advantage here is code simplicity. --- .../shuffle/sort/ShuffleExternalSorter.java | 3 +- .../spark/unsafe/map/BytesToBytesMap.java | 3 +- .../scala/org/apache/spark/Accumulators.scala | 19 +- .../scala/org/apache/spark/CacheManager.scala | 4 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 8 + .../org/apache/spark/executor/Executor.scala | 1 - .../apache/spark/executor/TaskMetrics.scala | 284 +++++++++--------- .../org/apache/spark/rdd/HadoopRDD.scala | 28 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 29 +- .../org/apache/spark/util/JsonProtocol.scala | 19 +- .../collection/ExternalAppendOnlyMap.scala | 4 +- .../util/collection/ExternalSorter.scala | 2 +- .../storage/DiskBlockObjectWriterSuite.scala | 6 +- .../ui/jobs/JobProgressListenerSuite.scala | 9 +- .../apache/spark/util/JsonProtocolSuite.scala | 5 +- .../datasources/SqlNewHadoopRDD.scala | 29 +- .../spark/tools/StoragePerfTester.scala | 2 +- 17 files changed, 236 insertions(+), 219 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 223b91854ef6..9affff80143d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -31,7 +31,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.ShuffleWriteMetrics$; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.DummySerializerInstance; @@ -139,7 +138,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // We're spilling, so bytes written should be counted towards spill rather than write. // Create a dummy WriteMetrics object to absorb these metrics, since we don't want to count // them towards shuffle bytes written. - writeMetricsToUse = ShuffleWriteMetrics$.MODULE$.createDummy(); + writeMetricsToUse = new ShuffleWriteMetrics(); } // This call performs the actual sort. diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 665893227d21..3387f9a4177c 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -30,7 +30,6 @@ import org.apache.spark.SparkEnv; import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.ShuffleWriteMetrics$; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.storage.BlockManager; @@ -341,7 +340,7 @@ public long spill(long numBytes) throws IOException { } // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = ShuffleWriteMetrics$.MODULE$.createDummy(); + ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); long released = 0L; while (dataPages.size() > 0) { diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index ea31f92d5330..8a9736774f70 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -387,6 +387,12 @@ private[spark] object InternalAccumulator { val RECORDS_WRITTEN = "metrics.output.recordsWritten" } + // Names of input metrics + object input { + val BYTES_READ = "metrics.input.bytesRead" + val RECORDS_READ = "metrics.input.recordsRead" + } + /** * Create a new internal Long accumulator with the specified name. */ @@ -412,6 +418,7 @@ private[spark] object InternalAccumulator { metricNames.map(newMetric) ++ createShuffleReadAccums() ++ createShuffleWriteAccums() ++ + createInputAccums() ++ createOutputAccums() } @@ -440,14 +447,20 @@ private[spark] object InternalAccumulator { shuffleWrite.SHUFFLE_WRITE_TIME).map(newMetric) } + /** + * Accumulators for tracking input metrics. + * Note: this method does not register accumulators for cleanup. + */ + def createInputAccums(): Seq[Accumulator[Long]] = { + Seq[String](input.BYTES_READ, input.RECORDS_READ).map(newMetric) + } + /** * Accumulators for tracking output metrics. * Note: this method does not register accumulators for cleanup. */ private def createOutputAccums(): Seq[Accumulator[Long]] = { - Seq[String]( - output.BYTES_WRITTEN, - output.RECORDS_WRITTEN).map(newMetric) + Seq[String](output.BYTES_WRITTEN, output.RECORDS_WRITTEN).map(newMetric) } /** diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 4d20c7369376..a4d129a6e132 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -45,8 +45,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { case Some(blockResult) => // Partition is already materialized, so just return its values val existingMetrics = context.taskMetrics - .getInputMetricsForReadMethod(blockResult.readMethod) - existingMetrics.incBytesRead(blockResult.bytes) + .registerInputMetrics(blockResult.readMethod) + existingMetrics.setBytesRead(blockResult.bytes) val iter = blockResult.data.asInstanceOf[Iterator[T]] new InterruptibleIterator[T](context, iter) { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 4bd94f13e57e..300bc5e7cb91 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -370,6 +370,14 @@ object SparkHadoopUtil { val SPARK_YARN_CREDS_COUNTER_DELIM = "-" + /** + * Number of records to update input metrics when reading from HadoopRDDs. + * + * Each update is potentially expensive because we need to use reflection to access the Hadoop + * FileSystem API of interest (it is only available in 2.5), so we should do it sparingly. + */ + private[spark] val UPDATE_INPUT_METRICS_INTERVAL_RECORDS = 1000 + def get: SparkHadoopUtil = { // Check each time to support changing to/from YARN val yarnMode = java.lang.Boolean.valueOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 75d7e34d60eb..425b8370ae9b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -426,7 +426,6 @@ private[spark] class Executor( if (taskRunner.task != null) { taskRunner.task.metrics.foreach { metrics => metrics.mergeShuffleReadMetrics() - metrics.updateInputMetrics() metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) metrics.updateAccumulators() diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index fe12f0bc2743..e1ef6a85569d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -127,6 +127,66 @@ class TaskMetrics( private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + /* ============================ * + | OUTPUT METRICS | + * ============================ */ + + private var _outputMetrics: Option[OutputMetrics] = None + + /** + * Metrics related to writing data externally (e.g. to a distributed filesystem), + * defined only in tasks with output. + */ + def outputMetrics: Option[OutputMetrics] = _outputMetrics + + /** + * Get or create a new [[OutputMetrics]] associated with this task. + */ + def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = synchronized { + _outputMetrics.getOrElse { + val metrics = new OutputMetrics(writeMethod, accumMap) + _outputMetrics = Some(metrics) + metrics + } + } + + + /* ========================== * + | INPUT METRICS | + * ========================== */ + + private var _inputMetrics: Option[InputMetrics] = None + + /** + * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted + * data, defined only in tasks with input. + */ + def inputMetrics: Option[InputMetrics] = _inputMetrics + + /** + * Get or create a new [[InputMetrics]] associated with this task. + */ + private[spark] def registerInputMetrics(readMethod: DataReadMethod): InputMetrics = { + synchronized { + val metrics = _inputMetrics.getOrElse { + val metrics = new InputMetrics(readMethod, accumMap) + _inputMetrics = Some(metrics) + metrics + } + // If there already exists an InputMetric with the same read method, we can just return + // that one. Otherwise, if the read method is different from the one previously seen by + // this task, we return a new dummy one to avoid clobbering the values of the old metrics. + // In the future we should try to store input metrics from all different read methods at + // the same time (SPARK-5225). + if (metrics.readMethod == readMethod) { + metrics + } else { + new InputMetrics(readMethod) + } + } + } + + /* =================================== * | SHUFFLE WRITE METRICS | * =================================== */ @@ -141,7 +201,7 @@ class TaskMetrics( /** * Get or create a new [[ShuffleWriteMetrics]] associated with this task. */ - def registerShuffleWriteMetrics(): ShuffleWriteMetrics = { + def registerShuffleWriteMetrics(): ShuffleWriteMetrics = synchronized { _shuffleWriteMetrics.getOrElse { val metrics = new ShuffleWriteMetrics(accumMap) _shuffleWriteMetrics = Some(metrics) @@ -149,6 +209,7 @@ class TaskMetrics( } } + /* ================================== * | SHUFFLE READ METRICS | * ================================== */ @@ -177,7 +238,7 @@ class TaskMetrics( * which merges the temporary values synchronously. */ private[spark] def registerTempShuffleReadMetrics(): ShuffleReadMetrics = synchronized { - val readMetrics = ShuffleReadMetrics.createDummy() + val readMetrics = new ShuffleReadMetrics tempShuffleReadMetrics += readMetrics readMetrics } @@ -200,91 +261,16 @@ class TaskMetrics( agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) } - /* ============================ * - | OUTPUT METRICS | - * ============================ */ - - private var _outputMetrics: Option[OutputMetrics] = None - - /** - * Metrics related to writing data externally (e.g. to a distributed filesystem), - * defined only tasks with output. - */ - def outputMetrics: Option[OutputMetrics] = _outputMetrics - - /** - * Get or create a new [[OutputMetrics]] associated with this task. - */ - def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = { - _outputMetrics.getOrElse { - val metrics = new OutputMetrics(writeMethod, accumMap) - _outputMetrics = Some(metrics) - metrics - } - } - - - - - - - - /* ================================== * | OTHER THINGS... WIP | * ================================== */ - /** - * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read - * are stored here. - */ - private var _inputMetrics: Option[InputMetrics] = None - - def inputMetrics: Option[InputMetrics] = _inputMetrics - - /** - * This should only be used when recreating TaskMetrics, not when updating input metrics in - * executors - */ - private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { - _inputMetrics = inputMetrics - } - /** * Storage statuses of any blocks that have been updated as a result of this task. */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - /** - * Returns the input metrics object that the task should use. Currently, if - * there exists an input metric with the same readMethod, we return that one - * so the caller can accumulate bytes read. If the readMethod is different - * than previously seen by this task, we return a new InputMetric but don't - * record it. - * - * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, - * we can store all the different inputMetrics (one per readMethod). - */ - private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): InputMetrics = { - synchronized { - _inputMetrics match { - case None => - val metrics = new InputMetrics(readMethod) - _inputMetrics = Some(metrics) - metrics - case Some(metrics @ InputMetrics(method)) if method == readMethod => - metrics - case Some(InputMetrics(method)) => - new InputMetrics(readMethod) - } - } - } - - private[spark] def updateInputMetrics(): Unit = synchronized { - inputMetrics.foreach(_.updateBytesRead()) - } - private var _accumulatorUpdates: Map[Long, Any] = Map.empty @transient private var _accumulatorsUpdater: () => Map[Long, Any] = null @@ -303,29 +289,11 @@ class TaskMetrics( } -private[spark] object TaskMetrics { - private val hostNameCache = new ConcurrentHashMap[String, String]() - - def empty: TaskMetrics = new TaskMetrics - - /** - * Get the hostname from cached data, since hostname is the order of number of nodes in cluster, - * so using cached hostname will decrease the object number and alleviate the GC overhead. - */ - def getCachedHostName: String = { - val host = Utils.localHostName() - val canonicalHost = hostNameCache.putIfAbsent(host, host) - if (canonicalHost != null) canonicalHost else host - } -} - - /** - * :: DeveloperApi :: * Method by which input data was read. Network means that the data was read over the network * from a remote block manager (which may have stored the data on-disk or in-memory). */ -@DeveloperApi +@deprecated("DataReadMethod will be made private in a future version.", "2.0.0") object DataReadMethod extends Enumeration with Serializable { type DataReadMethod = Value val Memory, Disk, Hadoop, Network = Value @@ -333,10 +301,9 @@ object DataReadMethod extends Enumeration with Serializable { /** - * :: DeveloperApi :: * Method by which output data was written. */ -@DeveloperApi +@deprecated("DataWriteMethod will be made private in a future version.", "2.0.0") object DataWriteMethod extends Enumeration with Serializable { type DataWriteMethod = Value val Hadoop = Value @@ -344,47 +311,51 @@ object DataWriteMethod extends Enumeration with Serializable { /** - * :: DeveloperApi :: * Metrics about reading input data. */ -@DeveloperApi -case class InputMetrics(readMethod: DataReadMethod.Value) { +@deprecated("InputMetrics will be made private in a future version.", "2.0.0") +class InputMetrics private ( + val readMethod: DataReadMethod.Value, + _bytesRead: Accumulator[Long], + _recordsRead: Accumulator[Long]) + extends Serializable { - /** - * This is volatile so that it is visible to the updater thread. - */ - @volatile @transient var bytesReadCallback: Option[() => Long] = None + private[executor] def this( + readMethod: DataReadMethod.Value, + accumMap: Map[String, Accumulator[Long]]) { + this( + readMethod, + accumMap(InternalAccumulator.input.BYTES_READ), + accumMap(InternalAccumulator.input.RECORDS_READ)) + } /** - * Total bytes read. + * Create a new [[InputMetrics]] that is not associated with any particular task. + * + * This mainly exists because of SPARK-5225, where we are forced to use a dummy [[InputMetrics]] + * because we want to ignore metrics from a second read method. In the future, we should revisit + * whether this is needed. + * + * A better alternative to use is [[TaskMetrics.registerInputMetrics]]. */ - private var _bytesRead: Long = _ - def bytesRead: Long = _bytesRead - def incBytesRead(bytes: Long): Unit = _bytesRead += bytes + private[spark] def this(readMethod: DataReadMethod.Value) { + this( + readMethod, + InternalAccumulator.createInputAccums().map { acc => (acc.name.get, acc) }.toMap) + } /** - * Total records read. + * Total number of bytes read. */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - def incRecordsRead(records: Long): Unit = _recordsRead += records + def bytesRead: Long = _bytesRead.value /** - * Invoke the bytesReadCallback and mutate bytesRead. + * Total number of records read. */ - def updateBytesRead() { - bytesReadCallback.foreach { c => - _bytesRead = c() - } - } + def recordsRead: Long = _recordsRead.value - /** - * Register a function that can be called to get up-to-date information on how many bytes the task - * has read from an input source. - */ - def setBytesReadCallback(f: Option[() => Long]) { - bytesReadCallback = f - } + private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) } @@ -445,6 +416,19 @@ class ShuffleReadMetrics private ( accumMap(InternalAccumulator.shuffleRead.RECORDS_READ)) } + /** + * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + * + * * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. + */ + private[spark] def this() { + this(InternalAccumulator.createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap) + } + /** * Number of remote blocks fetched in this shuffle by this task. */ @@ -502,21 +486,6 @@ class ShuffleReadMetrics private ( private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) } -private[spark] object ShuffleReadMetrics { - - /** - * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. - */ - def createDummy(): ShuffleReadMetrics = { - new ShuffleReadMetrics( - InternalAccumulator.createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap) - } -} - /** * Metrics pertaining to shuffle data written in a given task. @@ -535,6 +504,19 @@ class ShuffleWriteMetrics private ( accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_WRITE_TIME)) } + /** + * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + * + * A better alternative to use is [[TaskMetrics.registerShuffleWriteMetrics]]. + */ + private[spark] def this() { + this(InternalAccumulator.createShuffleWriteAccums().map { acc => (acc.name.get, acc) }.toMap) + } + /** * Number of bytes written for the shuffle by this task. */ @@ -563,17 +545,19 @@ class ShuffleWriteMetrics private ( } } -private[spark] object ShuffleWriteMetrics { + +private[spark] object TaskMetrics { + private val hostNameCache = new ConcurrentHashMap[String, String]() + + def empty: TaskMetrics = new TaskMetrics /** - * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. + * Get the hostname from cached data, since hostname is the order of number of nodes in cluster, + * so using cached hostname will decrease the object number and alleviate the GC overhead. */ - def createDummy(): ShuffleWriteMetrics = { - new ShuffleWriteMetrics( - InternalAccumulator.createShuffleWriteAccums().map { acc => (acc.name.get, acc) }.toMap) + def getCachedHostName: String = { + val host = Utils.localHostName() + val canonicalHost = hostNameCache.putIfAbsent(host, host) + if (canonicalHost != null) canonicalHost else host } } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 920d3bf219ff..10018c200cf7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -212,7 +212,7 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - val inputMetrics = context.taskMetrics.getInputMetricsForReadMethod(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) // Sets the thread local variable for the file's name split.inputSplit.value match { @@ -220,16 +220,21 @@ class HadoopRDD[K, V]( case _ => SqlNewHadoopRDDState.unsetInputFileName() } + // TODO: duplicate code alert! + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { - split.inputSplit.value match { - case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - case _ => None + val getBytesReadCallback: Option[() => Long] = split.inputSplit.value match { + case _: FileSplit | _: CombineFileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + case _ => None + } + + def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(getBytesRead()) } } - inputMetrics.setBytesReadCallback(bytesReadCallback) var reader: RecordReader[K, V] = null val inputFormat = getInputFormat(jobConf) @@ -252,6 +257,9 @@ class HadoopRDD[K, V]( if (!finished) { inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } (key, value) } @@ -272,14 +280,14 @@ class HadoopRDD[K, V]( } finally { reader = null } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() + if (getBytesReadCallback.isDefined) { + updateBytesRead() } else if (split.inputSplit.value.isInstanceOf[FileSplit] || split.inputSplit.value.isInstanceOf[CombineFileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.incBytesRead(split.inputSplit.value.getLength) + inputMetrics.setBytesRead(split.inputSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 8b330a34c3d3..ec49ba45eec0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -122,19 +122,23 @@ class NewHadoopRDD[K, V]( logInfo("Input split: " + split.serializableHadoopSplit) val conf = getConf - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) + + // TODO: duplicate code alert! // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { - split.serializableHadoopSplit.value match { - case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - case _ => None + val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { + case _: FileSplit | _: CombineFileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + case _ => None + } + + def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(getBytesRead()) } } - inputMetrics.setBytesReadCallback(bytesReadCallback) val format = inputFormatClass.newInstance format match { @@ -176,6 +180,9 @@ class NewHadoopRDD[K, V]( if (!finished) { inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } (reader.getCurrentKey, reader.getCurrentValue) } @@ -195,14 +202,14 @@ class NewHadoopRDD[K, V]( } finally { reader = null } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() + if (getBytesReadCallback.isDefined) { + updateBytesRead() } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] || split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) + inputMetrics.setBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 1ec797fa7983..4c80ff555186 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -744,8 +744,14 @@ private[spark] object JsonProtocol { outputMetrics.setRecordsWritten((outJson \ "Records Written").extract[Long]) } - metrics.setInputMetrics( - Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) + // Input metrics + Utils.jsonOption(json \ "Input Metrics").foreach { inJson => + val readMethod = DataReadMethod.withName((json \ "Data Read Method").extract[String]) + val inputMetrics = metrics.registerInputMetrics(readMethod) + inputMetrics.setBytesRead((json \ "Bytes Read").extract[Long]) + inputMetrics.incRecordsRead((json \ "Records Read").extract[Long]) + } + metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => value.extract[List[JValue]].map { block => @@ -757,15 +763,6 @@ private[spark] object JsonProtocol { metrics } - // TODO: kill this method - private def inputMetricsFromJson(json: JValue): InputMetrics = { - val metrics = new InputMetrics( - DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.incBytesRead((json \ "Bytes Read").extract[Long]) - metrics.incRecordsRead((json \ "Records Read").extractOpt[Long].getOrElse(0)) - metrics - } - def taskEndReasonFromJson(json: JValue): TaskEndReason = { val success = Utils.getFormattedClassName(Success) val resubmitted = Utils.getFormattedClassName(Resubmitted) 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 7cb9444de784..6f2e5c971d30 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 @@ -181,7 +181,7 @@ class ExternalAppendOnlyMap[K, V, C]( */ override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempLocalBlock() - curWriteMetrics = ShuffleWriteMetrics.createDummy() + curWriteMetrics = new ShuffleWriteMetrics var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 @@ -208,7 +208,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - curWriteMetrics = ShuffleWriteMetrics.createDummy() + curWriteMetrics = new ShuffleWriteMetrics writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index cfd4866c0e36..6e7959ae55e3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -245,7 +245,7 @@ private[spark] class ExternalSorter[K, V, C]( var writer: DiskBlockObjectWriter = null def openWriter(): Unit = { assert (writer == null && spillMetrics == null) - spillMetrics = ShuffleWriteMetrics.createDummy() + spillMetrics = new ShuffleWriteMetrics writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics) } openWriter() diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index bd0978ff259b..e6d752075cea 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -44,7 +44,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("verify write metrics") { val file = new File(tempDir, "somefile") - val writeMetrics = ShuffleWriteMetrics.createDummy() + val writeMetrics = new ShuffleWriteMetrics val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) @@ -118,7 +118,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("commitAndClose() should be idempotent") { val file = new File(tempDir, "somefile") - val writeMetrics = ShuffleWriteMetrics.createDummy() + val writeMetrics = new ShuffleWriteMetrics val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) for (i <- 1 to 1000) { @@ -136,7 +136,7 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { test("revertPartialWritesAndClose() should be idempotent") { val file = new File(tempDir, "somefile") - val writeMetrics = ShuffleWriteMetrics.createDummy() + val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter( file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) for (i <- 1 to 1000) { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 15af7dd22026..0f80de52a15f 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -272,7 +272,8 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val taskMetrics = new TaskMetrics() val shuffleReadMetrics = taskMetrics.registerTempShuffleReadMetrics() val shuffleWriteMetrics = taskMetrics.registerShuffleWriteMetrics() - taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) + val inputMetrics = taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) + val outputMetrics = taskMetrics.registerOutputMetrics(DataWriteMethod.Hadoop) shuffleReadMetrics.incRemoteBytesRead(base + 1) shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) @@ -281,11 +282,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with taskMetrics.setExecutorRunTime(base + 4) taskMetrics.incDiskBytesSpilled(base + 5) taskMetrics.incMemoryBytesSpilled(base + 6) - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - taskMetrics.setInputMetrics(Some(inputMetrics)) - inputMetrics.incBytesRead(base + 7) - val outputMetrics = taskMetrics.registerOutputMetrics(DataWriteMethod.Hadoop) - taskMetrics.outputMetrics = Some(outputMetrics) + inputMetrics.setBytesRead(base + 7) outputMetrics.setBytesWritten(base + 8) taskMetrics } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e4deeece4e35..6f3020b5cda1 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -772,10 +772,9 @@ class JsonProtocolSuite extends SparkFunSuite { t.incMemoryBytesSpilled(a + c) if (hasHadoopInput) { - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - inputMetrics.incBytesRead(d + e + f) + val inputMetrics = t.registerInputMetrics(DataReadMethod.Hadoop) + inputMetrics.setBytesRead(d + e + f) inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1) - t.setInputMetrics(Some(inputMetrics)) } else { val sr = t.registerTempShuffleReadMetrics() sr.incRemoteBytesRead(b + d) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index 12f8783f846d..8918597df531 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala @@ -126,8 +126,7 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( logInfo("Input split: " + split.serializableHadoopSplit) val conf = getConf(isDriverSide = false) - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) // Sets the thread local variable for the file's name split.serializableHadoopSplit.value match { @@ -135,16 +134,21 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( case _ => SqlNewHadoopRDDState.unsetInputFileName() } + // TODO: duplicate code alert! + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { - split.serializableHadoopSplit.value match { - case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - case _ => None + val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { + case _: FileSplit | _: CombineFileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + case _ => None + } + + def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(getBytesRead()) } } - inputMetrics.setBytesReadCallback(bytesReadCallback) val format = inputFormatClass.newInstance format match { @@ -209,6 +213,9 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( if (!finished) { inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } reader.getCurrentValue } @@ -229,14 +236,14 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( } finally { reader = null } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() + if (getBytesReadCallback.isDefined) { + updateBytesRead() } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] || split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) + inputMetrics.setBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index b0cbf6f11155..2c63a870a721 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -61,7 +61,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf), ShuffleWriteMetrics.createDummy()) + new KryoSerializer(sc.conf), new ShuffleWriteMetrics) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeKey, writeValue) From 17becb05aa347586fe07fd452e288479c7649e82 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 16:06:31 -0800 Subject: [PATCH 09/65] Fix JsonProtocol + JsonProtocolSuite --- .../scala/org/apache/spark/util/JsonProtocol.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) 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 4c80ff555186..1603ae743fbc 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -724,7 +724,7 @@ private[spark] object JsonProtocol { readMetrics.setRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) readMetrics.setLocalBytesRead((readJson \ "Local Bytes Read").extractOpt[Long].getOrElse(0L)) readMetrics.setFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) - readMetrics.setRecordsRead((readJson \ "Total Records Read").extract[Long]) + readMetrics.setRecordsRead((readJson \ "Total Records Read").extractOpt[Long].getOrElse(0L)) metrics.mergeShuffleReadMetrics() } @@ -732,7 +732,8 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Write Metrics").foreach { writeJson => val writeMetrics = metrics.registerShuffleWriteMetrics() writeMetrics.incShuffleBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) - writeMetrics.incShuffleRecordsWritten((writeJson \ "Shuffle Records Written").extract[Long]) + writeMetrics.incShuffleRecordsWritten((writeJson \ "Shuffle Records Written") + .extractOpt[Long].getOrElse(0L)) writeMetrics.incShuffleWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) } @@ -741,15 +742,15 @@ private[spark] object JsonProtocol { val writeMethod = DataWriteMethod.withName((outJson \ "Data Write Method").extract[String]) val outputMetrics = metrics.registerOutputMetrics(writeMethod) outputMetrics.setBytesWritten((outJson \ "Bytes Written").extract[Long]) - outputMetrics.setRecordsWritten((outJson \ "Records Written").extract[Long]) + outputMetrics.setRecordsWritten((outJson \ "Records Written").extractOpt[Long].getOrElse(0L)) } // Input metrics Utils.jsonOption(json \ "Input Metrics").foreach { inJson => - val readMethod = DataReadMethod.withName((json \ "Data Read Method").extract[String]) + val readMethod = DataReadMethod.withName((inJson \ "Data Read Method").extract[String]) val inputMetrics = metrics.registerInputMetrics(readMethod) - inputMetrics.setBytesRead((json \ "Bytes Read").extract[Long]) - inputMetrics.incRecordsRead((json \ "Records Read").extract[Long]) + inputMetrics.setBytesRead((inJson \ "Bytes Read").extract[Long]) + inputMetrics.incRecordsRead((inJson \ "Records Read").extractOpt[Long].getOrElse(0L)) } metrics.updatedBlocks = From 809a93ac3b4961e35bfb180fa1e46bc36296153c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 16:25:22 -0800 Subject: [PATCH 10/65] Fix tests where TaskMetrics had no accumulators Tests are still failing as of this commit. E.g. SortShuffleSuite. --- .../main/scala/org/apache/spark/TaskContext.scala | 3 +-- .../scala/org/apache/spark/TaskContextImpl.scala | 15 ++++++++++++++- .../org/apache/spark/executor/TaskMetrics.scala | 8 ++++---- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index af558d6e5b47..fcd2093607c3 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -153,8 +153,7 @@ abstract class TaskContext extends Serializable { */ def taskAttemptId(): Long - /** ::DeveloperApi:: */ - @DeveloperApi + @deprecated("TaskMetrics will be made private in a future version", "2.0.0") def taskMetrics(): TaskMetrics /** diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 06b6555225ee..11a9f5261cf2 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -122,5 +122,18 @@ private[spark] class TaskContextImpl( internalAccumulators.map { a => (a.name.get, a) }.toMap } - val taskMetrics: TaskMetrics = new TaskMetrics(internalMetricsToAccumulators) + /** + * Metrics associated with this task. + */ + val taskMetrics: TaskMetrics = { + val testing = sys.props.contains("spark.testing") + if (testing && internalMetricsToAccumulators.isEmpty) { + // In tests, we may construct our own dummy TaskContexts where the list of internal + // accumulators is empty. Since TaskMetrics complains if it doesn't find the expected + // accumulators, we just pass in a list of dummy ones here. + new TaskMetrics(InternalAccumulator.create().map { a => (a.name.get, a) }.toMap) + } else { + new TaskMetrics(internalMetricsToAccumulators) + } + } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e1ef6a85569d..2ccf5b9f4694 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -48,7 +48,7 @@ class TaskMetrics( import InternalAccumulator._ def this(host: String) { - this(InternalAccumulator.create().map { accum => (accum.name.get, accum) }.toMap, host) + this(InternalAccumulator.create().map { a => (a.name.get, a) }.toMap, host) } // Needed for Java @@ -341,7 +341,7 @@ class InputMetrics private ( private[spark] def this(readMethod: DataReadMethod.Value) { this( readMethod, - InternalAccumulator.createInputAccums().map { acc => (acc.name.get, acc) }.toMap) + InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) } /** @@ -426,7 +426,7 @@ class ShuffleReadMetrics private ( * * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. */ private[spark] def this() { - this(InternalAccumulator.createShuffleReadAccums().map { acc => (acc.name.get, acc) }.toMap) + this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) } /** @@ -514,7 +514,7 @@ class ShuffleWriteMetrics private ( * A better alternative to use is [[TaskMetrics.registerShuffleWriteMetrics]]. */ private[spark] def this() { - this(InternalAccumulator.createShuffleWriteAccums().map { acc => (acc.name.get, acc) }.toMap) + this(InternalAccumulator.createShuffleWriteAccums().map { a => (a.name.get, a) }.toMap) } /** From 32ba9e3b78f679261e36b4ea095733a14b30fd7b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 16:29:18 -0800 Subject: [PATCH 11/65] Rename a few shuffle write metrics for consistency --- .../sort/BypassMergeSortShuffleWriter.java | 4 +- .../shuffle/sort/ShuffleExternalSorter.java | 4 +- .../shuffle/sort/UnsafeShuffleWriter.java | 7 ++- .../storage/TimeTrackingOutputStream.java | 10 ++-- .../scala/org/apache/spark/Accumulators.scala | 12 ++--- .../apache/spark/executor/TaskMetrics.scala | 29 ++++++------ .../spark/scheduler/SparkListener.scala | 2 +- .../shuffle/FileShuffleBlockResolver.scala | 2 +- .../shuffle/sort/SortShuffleWriter.scala | 2 +- .../status/api/v1/AllStagesResource.scala | 8 ++-- .../spark/storage/DiskBlockObjectWriter.scala | 12 ++--- .../apache/spark/ui/exec/ExecutorsTab.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 8 ++-- .../org/apache/spark/ui/jobs/StagePage.scala | 10 ++-- .../org/apache/spark/util/JsonProtocol.scala | 11 +++-- .../collection/ExternalAppendOnlyMap.scala | 4 +- .../util/collection/ExternalSorter.scala | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 20 ++++---- .../scala/org/apache/spark/ShuffleSuite.scala | 4 +- .../metrics/InputOutputMetricsSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../BypassMergeSortShuffleWriterSuite.scala | 8 ++-- .../storage/DiskBlockObjectWriterSuite.scala | 46 +++++++++---------- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 10 ++-- 25 files changed, 112 insertions(+), 113 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a7aae092bc22..3469251be36d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -142,7 +142,7 @@ public void write(Iterator> records) throws IOException { // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be // included in the shuffle write time. - writeMetrics.incShuffleWriteTime(System.nanoTime() - openStartTime); + writeMetrics.incWriteTime(System.nanoTime() - openStartTime); while (records.hasNext()) { final Product2 record = records.next(); @@ -199,7 +199,7 @@ private long[] writePartitionedFile(File outputFile) throws IOException { threwException = false; } finally { Closeables.close(out, threwException); - writeMetrics.incShuffleWriteTime(System.nanoTime() - writeStartTime); + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); } partitionWriters = null; return lengths; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 9affff80143d..2c84de5bf2a5 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -233,8 +233,8 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // Note that we intentionally ignore the value of `writeMetricsToUse.shuffleWriteTime()`. // Consistent with ExternalSorter, we do not count this IO towards shuffle write time. // This means that this IO time is not accounted for anywhere; SPARK-3577 will fix this. - writeMetrics.incShuffleRecordsWritten(writeMetricsToUse.shuffleRecordsWritten()); - taskContext.taskMetrics().incDiskBytesSpilled(writeMetricsToUse.shuffleBytesWritten()); + writeMetrics.incRecordsWritten(writeMetricsToUse.recordsWritten()); + taskContext.taskMetrics().incDiskBytesSpilled(writeMetricsToUse.bytesWritten()); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index b1c8c227d315..d49e2b62b55b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -25,7 +25,6 @@ import scala.Option; import scala.Product2; import scala.collection.JavaConverters; -import scala.collection.immutable.Map; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -297,8 +296,8 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. - writeMetrics.decShuffleBytesWritten(spills[spills.length - 1].file.length()); - writeMetrics.incShuffleBytesWritten(outputFile.length()); + writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); + writeMetrics.incBytesWritten(outputFile.length()); return partitionLengths; } } catch (IOException e) { @@ -410,7 +409,7 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th spillInputChannelPositions[i] += actualBytesTransferred; bytesToTransfer -= actualBytesTransferred; } - writeMetrics.incShuffleWriteTime(System.nanoTime() - writeStartTime); + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); bytesWrittenToMergedFile += partitionLengthInSpill; partitionLengths[partition] += partitionLengthInSpill; } diff --git a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java index dc2aa30466cc..5d0555a8c28e 100644 --- a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java +++ b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java @@ -42,34 +42,34 @@ public TimeTrackingOutputStream(ShuffleWriteMetrics writeMetrics, OutputStream o public void write(int b) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void write(byte[] b) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void write(byte[] b, int off, int len) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b, off, len); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void flush() throws IOException { final long startTime = System.nanoTime(); outputStream.flush(); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void close() throws IOException { final long startTime = System.nanoTime(); outputStream.close(); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } } diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 8a9736774f70..5ca0cc9d7d45 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -376,9 +376,9 @@ private[spark] object InternalAccumulator { // Names of shuffle write metrics object shuffleWrite { - val SHUFFLE_BYTES_WRITTEN = "metrics.shuffle.write.shuffleBytesWritten" - val SHUFFLE_RECORDS_WRITTEN = "metrics.shuffle.write.shuffleRecordsWritten" - val SHUFFLE_WRITE_TIME = "metrics.shuffle.write.shuffleWriteTime" + val BYTES_WRITTEN = "metrics.shuffle.write.shuffleBytesWritten" + val RECORDS_WRITTEN = "metrics.shuffle.write.shuffleRecordsWritten" + val WRITE_TIME = "metrics.shuffle.write.shuffleWriteTime" } // Names of output metrics @@ -442,9 +442,9 @@ private[spark] object InternalAccumulator { */ def createShuffleWriteAccums(): Seq[Accumulator[Long]] = { Seq[String]( - shuffleWrite.SHUFFLE_BYTES_WRITTEN, - shuffleWrite.SHUFFLE_RECORDS_WRITTEN, - shuffleWrite.SHUFFLE_WRITE_TIME).map(newMetric) + shuffleWrite.BYTES_WRITTEN, + shuffleWrite.RECORDS_WRITTEN, + shuffleWrite.WRITE_TIME).map(newMetric) } /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 2ccf5b9f4694..9e646a4b6165 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -22,7 +22,6 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulator, InternalAccumulator} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils @@ -492,16 +491,16 @@ class ShuffleReadMetrics private ( */ @deprecated("ShuffleWriteMetrics will be made private in a future version.", "2.0.0") class ShuffleWriteMetrics private ( - _shuffleBytesWritten: Accumulator[Long], - _shuffleRecordsWritten: Accumulator[Long], + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long], _shuffleWriteTime: Accumulator[Long]) extends Serializable { private[executor] def this(accumMap: Map[String, Accumulator[Long]]) { this( - accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_BYTES_WRITTEN), - accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_RECORDS_WRITTEN), - accumMap(InternalAccumulator.shuffleWrite.SHUFFLE_WRITE_TIME)) + accumMap(InternalAccumulator.shuffleWrite.BYTES_WRITTEN), + accumMap(InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), + accumMap(InternalAccumulator.shuffleWrite.WRITE_TIME)) } /** @@ -520,12 +519,12 @@ class ShuffleWriteMetrics private ( /** * Number of bytes written for the shuffle by this task. */ - def shuffleBytesWritten: Long = _shuffleBytesWritten.value + def bytesWritten: Long = _bytesWritten.value /** * Total number of records written to the shuffle by this task. */ - def shuffleRecordsWritten: Long = _shuffleRecordsWritten.value + def recordsWritten: Long = _recordsWritten.value /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. @@ -534,14 +533,14 @@ class ShuffleWriteMetrics private ( // TODO: these are not thread-safe. Is that OK? - private[spark] def incShuffleBytesWritten(v: Long): Unit = _shuffleBytesWritten.add(v) - private[spark] def incShuffleRecordsWritten(v: Long): Unit = _shuffleRecordsWritten.add(v) - private[spark] def incShuffleWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) - private[spark] def decShuffleBytesWritten(v: Long): Unit = { - _shuffleBytesWritten.setValue(shuffleBytesWritten - v) + private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) + private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) + private[spark] def incWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) + private[spark] def decBytesWritten(v: Long): Unit = { + _bytesWritten.setValue(bytesWritten - v) } - private[spark] def decShuffleRecordsWritten(v: Long): Unit = { - _shuffleRecordsWritten.setValue(shuffleRecordsWritten - v) + private[spark] def decRecordsWritten(v: Long): Unit = { + _recordsWritten.setValue(recordsWritten - v) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 075a7f13172d..a3340772c38b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -271,7 +271,7 @@ class StatsReportListener extends SparkListener with Logging { // Shuffle write showBytesDistribution("shuffle bytes written:", - (_, metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten), taskInfoMetrics) + (_, metric) => metric.shuffleWriteMetrics.map(_.bytesWritten), taskInfoMetrics) // Fetch & I/O showMillisDistribution("fetch wait time:", diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index cc5f933393ad..29765cfc305b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -90,7 +90,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, so should be included in the shuffle write time. - writeMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) + writeMetrics.incWriteTime(System.nanoTime - openStartTime) override def releaseWriters(success: Boolean) { shuffleState.completedMapTasks.add(mapId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 896021a7b8b2..b2cf17e67ecb 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -91,7 +91,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val startTime = System.nanoTime() sorter.stop() context.taskMetrics.shuffleWriteMetrics.foreach( - _.incShuffleWriteTime(System.nanoTime - startTime)) + _.incWriteTime(System.nanoTime - startTime)) sorter = null } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 31b4dd7c0f42..a6f021259d38 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -214,8 +214,8 @@ private[v1] object AllStagesResource { raw.shuffleWriteMetrics } def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.shuffleBytesWritten), - writeRecords = submetricQuantiles(_.shuffleRecordsWritten), + writeBytes = submetricQuantiles(_.bytesWritten), + writeRecords = submetricQuantiles(_.recordsWritten), writeTime = submetricQuantiles(_.shuffleWriteTime) ) }.metricOption @@ -283,9 +283,9 @@ private[v1] object AllStagesResource { def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { new ShuffleWriteMetrics( - bytesWritten = internal.shuffleBytesWritten, + bytesWritten = internal.bytesWritten, writeTime = internal.shuffleWriteTime, - recordsWritten = internal.shuffleRecordsWritten + recordsWritten = internal.recordsWritten ) } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index e2dd80f24393..08d71e212885 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -102,7 +102,7 @@ private[spark] class DiskBlockObjectWriter( objOut.flush() val start = System.nanoTime() fos.getFD.sync() - writeMetrics.incShuffleWriteTime(System.nanoTime() - start) + writeMetrics.incWriteTime(System.nanoTime() - start) } } { objOut.close() @@ -132,7 +132,7 @@ private[spark] class DiskBlockObjectWriter( close() finalPosition = file.length() // In certain compression codecs, more bytes are written after close() is called - writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition) + writeMetrics.incBytesWritten(finalPosition - reportedPosition) } else { finalPosition = file.length() } @@ -152,8 +152,8 @@ private[spark] class DiskBlockObjectWriter( // truncating the file to its initial position. try { if (initialized) { - writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) - writeMetrics.decShuffleRecordsWritten(numRecordsWritten) + writeMetrics.decBytesWritten(reportedPosition - initialPosition) + writeMetrics.decRecordsWritten(numRecordsWritten) objOut.flush() bs.flush() close() @@ -201,7 +201,7 @@ private[spark] class DiskBlockObjectWriter( */ def recordWritten(): Unit = { numRecordsWritten += 1 - writeMetrics.incShuffleRecordsWritten(1) + writeMetrics.incRecordsWritten(1) if (numRecordsWritten % 32 == 0) { updateBytesWritten() @@ -226,7 +226,7 @@ private[spark] class DiskBlockObjectWriter( */ private def updateBytesWritten() { val pos = channel.position() - writeMetrics.incShuffleBytesWritten(pos - reportedPosition) + writeMetrics.incBytesWritten(pos - reportedPosition) reportedPosition = pos } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index a88fc4c37d3c..67fd9a16e8ff 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -129,7 +129,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp } metrics.shuffleWriteMetrics.foreach { shuffleWrite => executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.bytesWritten } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ca37829216f2..4a9f8b30525f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -426,14 +426,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) val shuffleWriteDelta = - (taskMetrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L)) + (taskMetrics.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.bytesWritten).getOrElse(0L)) stageData.shuffleWriteBytes += shuffleWriteDelta execSummary.shuffleWrite += shuffleWriteDelta val shuffleWriteRecordsDelta = - (taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleRecordsWritten).getOrElse(0L)) + (taskMetrics.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.recordsWritten).getOrElse(0L)) stageData.shuffleWriteRecords += shuffleWriteRecordsDelta execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta 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 4c9b4f867b7d..32fc3e0e4b05 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 @@ -494,11 +494,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { getFormattedSizeQuantiles(shuffleReadRemoteSizes) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + metrics.get.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L).toDouble } val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L).toDouble + metrics.get.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = Shuffle Write Size / Records +: @@ -921,11 +921,11 @@ private[ui] class TaskDataSource( val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("") val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics) - val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten).getOrElse(0L) + val shuffleWriteSortable = maybeShuffleWrite.map(_.bytesWritten).getOrElse(0L) val shuffleWriteReadable = maybeShuffleWrite - .map(m => s"${Utils.bytesToString(m.shuffleBytesWritten)}").getOrElse("") + .map(m => s"${Utils.bytesToString(m.bytesWritten)}").getOrElse("") val shuffleWriteRecords = maybeShuffleWrite - .map(_.shuffleRecordsWritten.toString).getOrElse("") + .map(_.recordsWritten.toString).getOrElse("") val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.getOrElse(0L) 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 1603ae743fbc..f7770200777a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -303,9 +303,9 @@ private[spark] object JsonProtocol { }.getOrElse(JNothing) val shuffleWriteMetrics: JValue = taskMetrics.shuffleWriteMetrics.map { wm => - ("Shuffle Bytes Written" -> wm.shuffleBytesWritten) ~ + ("Shuffle Bytes Written" -> wm.bytesWritten) ~ ("Shuffle Write Time" -> wm.shuffleWriteTime) ~ - ("Shuffle Records Written" -> wm.shuffleRecordsWritten) + ("Shuffle Records Written" -> wm.recordsWritten) }.getOrElse(JNothing) val inputMetrics: JValue = taskMetrics.inputMetrics.map { im => @@ -729,12 +729,13 @@ private[spark] object JsonProtocol { } // Shuffle write metrics + // TODO: the fields don't have to say "Shuffle" again for consistency Utils.jsonOption(json \ "Shuffle Write Metrics").foreach { writeJson => val writeMetrics = metrics.registerShuffleWriteMetrics() - writeMetrics.incShuffleBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) - writeMetrics.incShuffleRecordsWritten((writeJson \ "Shuffle Records Written") + writeMetrics.incBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) + writeMetrics.incRecordsWritten((writeJson \ "Shuffle Records Written") .extractOpt[Long].getOrElse(0L)) - writeMetrics.incShuffleWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) } // Output metrics 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 6f2e5c971d30..fea1fd5a638d 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 @@ -193,8 +193,8 @@ class ExternalAppendOnlyMap[K, V, C]( val w = writer writer = null w.commitAndClose() - _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten - batchSizes.append(curWriteMetrics.shuffleBytesWritten) + _diskBytesSpilled += curWriteMetrics.bytesWritten + batchSizes.append(curWriteMetrics.bytesWritten) objectsWritten = 0 } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6e7959ae55e3..fdfbe08997da 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -262,8 +262,8 @@ private[spark] class ExternalSorter[K, V, C]( val w = writer writer = null w.commitAndClose() - _diskBytesSpilled += spillMetrics.shuffleBytesWritten - batchSizes.append(spillMetrics.shuffleBytesWritten) + _diskBytesSpilled += spillMetrics.bytesWritten + batchSizes.append(spillMetrics.bytesWritten) spillMetrics = null objectsWritten = 0 } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 5fe64bde3604..625fdd57eb5d 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -279,8 +279,8 @@ public void writeEmptyIterator() throws Exception { assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); - assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleRecordsWritten()); - assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleBytesWritten()); + assertEquals(0, taskMetrics.shuffleWriteMetrics().get().recordsWritten()); + assertEquals(0, taskMetrics.shuffleWriteMetrics().get().bytesWritten()); assertEquals(0, taskMetrics.diskBytesSpilled()); assertEquals(0, taskMetrics.memoryBytesSpilled()); } @@ -311,10 +311,10 @@ public void writeWithoutSpilling() throws Exception { HashMultiset.create(readRecordsFromFile())); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertEquals(0, taskMetrics.diskBytesSpilled()); assertEquals(0, taskMetrics.memoryBytesSpilled()); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } private void testMergingSpills( @@ -354,11 +354,11 @@ private void testMergingSpills( assertEquals(HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test @@ -416,11 +416,11 @@ public void writeEnoughDataToTriggerSpill() throws Exception { readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test @@ -437,11 +437,11 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 0de10ae48537..1de013a04e93 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -450,8 +450,8 @@ object ShuffleSuite { val listener = new SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { taskEnd.taskMetrics.shuffleWriteMetrics.foreach { m => - recordsWritten += m.shuffleRecordsWritten - bytesWritten += m.shuffleBytesWritten + recordsWritten += m.recordsWritten + bytesWritten += m.bytesWritten } taskEnd.taskMetrics.shuffleReadMetrics.foreach { m => recordsRead += m.recordsRead diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 44eb5a046912..6f2f8ea8a1c3 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -212,7 +212,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext metrics.inputMetrics.foreach(inputRead += _.recordsRead) metrics.outputMetrics.foreach(outputWritten += _.recordsWritten) metrics.shuffleReadMetrics.foreach(shuffleRead += _.recordsRead) - metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.shuffleRecordsWritten) + metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.recordsWritten) } }) 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 f20d5be7c0ee..715959c47882 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -270,7 +270,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match taskMetrics.inputMetrics should not be ('defined) taskMetrics.outputMetrics should not be ('defined) taskMetrics.shuffleWriteMetrics should be ('defined) - taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0L) + taskMetrics.shuffleWriteMetrics.get.bytesWritten should be > (0L) } if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index bb331bb385df..7a2b53f2b518 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -145,8 +145,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(outputFile.length() === 0) assert(temporaryFilesCreated.isEmpty) val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get - assert(shuffleWriteMetrics.shuffleBytesWritten === 0) - assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) + assert(shuffleWriteMetrics.bytesWritten === 0) + assert(shuffleWriteMetrics.recordsWritten === 0) assert(taskMetrics.diskBytesSpilled === 0) assert(taskMetrics.memoryBytesSpilled === 0) } @@ -168,8 +168,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(writer.getPartitionLengths.sum === outputFile.length()) assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get - assert(shuffleWriteMetrics.shuffleBytesWritten === outputFile.length()) - assert(shuffleWriteMetrics.shuffleRecordsWritten === records.length) + assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) + assert(shuffleWriteMetrics.recordsWritten === records.length) assert(taskMetrics.diskBytesSpilled === 0) assert(taskMetrics.memoryBytesSpilled === 0) } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index e6d752075cea..1d37eb3c7386 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -50,18 +50,18 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write - assert(writeMetrics.shuffleRecordsWritten === 1) + assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write - assert(writeMetrics.shuffleBytesWritten == 0) + assert(writeMetrics.bytesWritten == 0) // After 32 writes, metrics should update for (i <- 0 until 32) { writer.flush() writer.write(Long.box(i), Long.box(i)) } - assert(writeMetrics.shuffleBytesWritten > 0) - assert(writeMetrics.shuffleRecordsWritten === 33) + assert(writeMetrics.bytesWritten > 0) + assert(writeMetrics.recordsWritten === 33) writer.commitAndClose() - assert(file.length() == writeMetrics.shuffleBytesWritten) + assert(file.length() == writeMetrics.bytesWritten) } test("verify write metrics on revert") { @@ -72,19 +72,19 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write - assert(writeMetrics.shuffleRecordsWritten === 1) + assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write - assert(writeMetrics.shuffleBytesWritten == 0) + assert(writeMetrics.bytesWritten == 0) // After 32 writes, metrics should update for (i <- 0 until 32) { writer.flush() writer.write(Long.box(i), Long.box(i)) } - assert(writeMetrics.shuffleBytesWritten > 0) - assert(writeMetrics.shuffleRecordsWritten === 33) + assert(writeMetrics.bytesWritten > 0) + assert(writeMetrics.recordsWritten === 33) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleBytesWritten == 0) - assert(writeMetrics.shuffleRecordsWritten == 0) + assert(writeMetrics.bytesWritten == 0) + assert(writeMetrics.recordsWritten == 0) } test("Reopening a closed block writer") { @@ -109,11 +109,11 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(i, i) } writer.commitAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - assert(writeMetrics.shuffleRecordsWritten === 1000) + val bytesWritten = writeMetrics.bytesWritten + assert(writeMetrics.recordsWritten === 1000) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleRecordsWritten === 1000) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) + assert(writeMetrics.recordsWritten === 1000) + assert(writeMetrics.bytesWritten === bytesWritten) } test("commitAndClose() should be idempotent") { @@ -125,12 +125,12 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(i, i) } writer.commitAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten + val bytesWritten = writeMetrics.bytesWritten val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 1000) + assert(writeMetrics.recordsWritten === 1000) writer.commitAndClose() - assert(writeMetrics.shuffleRecordsWritten === 1000) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) + assert(writeMetrics.recordsWritten === 1000) + assert(writeMetrics.bytesWritten === bytesWritten) assert(writeMetrics.shuffleWriteTime === writeTime) } @@ -143,12 +143,12 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { writer.write(i, i) } writer.revertPartialWritesAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten + val bytesWritten = writeMetrics.bytesWritten val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 0) + assert(writeMetrics.recordsWritten === 0) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleRecordsWritten === 0) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) + assert(writeMetrics.recordsWritten === 0) + assert(writeMetrics.bytesWritten === bytesWritten) assert(writeMetrics.shuffleWriteTime === writeTime) } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 0f80de52a15f..e6c13fb2c7c8 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -278,7 +278,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) taskMetrics.mergeShuffleReadMetrics() - shuffleWriteMetrics.incShuffleBytesWritten(base + 3) + shuffleWriteMetrics.incBytesWritten(base + 3) taskMetrics.setExecutorRunTime(base + 4) taskMetrics.incDiskBytesSpilled(base + 5) taskMetrics.incMemoryBytesSpilled(base + 6) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 6f3020b5cda1..affaec268a79 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -228,7 +228,7 @@ class JsonProtocolSuite extends SparkFunSuite { .removeField { case (field, _) => field == "Shuffle Records Written" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.shuffleReadMetrics.get.recordsRead == 0) - assert(newMetrics.shuffleWriteMetrics.get.shuffleRecordsWritten == 0) + assert(newMetrics.shuffleWriteMetrics.get.recordsWritten == 0) } test("OutputMetrics backward compatibility") { @@ -568,7 +568,7 @@ class JsonProtocolSuite extends SparkFunSuite { } private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { - assert(metrics1.shuffleBytesWritten === metrics2.shuffleBytesWritten) + assert(metrics1.bytesWritten === metrics2.bytesWritten) assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime) } @@ -791,9 +791,9 @@ class JsonProtocolSuite extends SparkFunSuite { outputMetrics.setRecordsWritten(if (hasRecords) (a + b + c)/100 else -1) } else { val sw = t.registerShuffleWriteMetrics() - sw.incShuffleBytesWritten(a + b + c) - sw.incShuffleWriteTime(b + c + d) - sw.incShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) + sw.incBytesWritten(a + b + c) + sw.incWriteTime(b + c + d) + sw.incRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) } // Make at most 6 blocks t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => From 78fb33e3b7714c3ffff932434000780d17835c24 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 18:06:10 -0800 Subject: [PATCH 12/65] Fix metrics in local mode (tests) Tests were previously failing because we end up double counting metrics in local mode. This is because each TaskContext shares the same list of accumulators, so they end up updating the metrics on top of each other. The fix is to ensure TaskContext clears any existing values on the accumulators before passing them on. --- .../scala/org/apache/spark/Accumulators.scala | 9 ++++++++- .../scala/org/apache/spark/TaskContextImpl.scala | 15 +++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5ca0cc9d7d45..bcbc1f21f682 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -69,7 +69,7 @@ class Accumulable[R, T] private[spark] ( val id: Long = Accumulators.newId() @volatile private var value_ : R = initialValue - val zero = param.zero(initialValue) // Zero value to be passed to workers + val zero = param.zero(initialValue) Accumulators.register(this) @@ -140,6 +140,13 @@ class Accumulable[R, T] private[spark] ( this.value = newValue } + /** + * Reset the accumulator's value to zero. + */ + private[spark] def resetValue(): Unit = { + setValue(zero) + } + // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 11a9f5261cf2..e5e67e7b6f2c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -116,10 +116,17 @@ private[spark] class TaskContextImpl( } private[spark] override val internalMetricsToAccumulators: Map[String, Accumulator[Long]] = { - // Explicitly register internal accumulators here because these are - // not captured in the task closure and are already deserialized - internalAccumulators.foreach(registerAccumulator) - internalAccumulators.map { a => (a.name.get, a) }.toMap + // Explicitly register internal accumulators here because these are not captured in the + // task closure and are already deserialized. Additionally, since the driver expects + // us to update these accumulators and report back partial values, we should reset + // each accumulator to zero. Otherwise, we may end up double counting in local mode. + internalAccumulators.map { a => + registerAccumulator(a) + a.resetValue() + assert(a.name.isDefined, "internal accumulator is expected to be named") + assert(a.isInternal, "internal accumulator is not marked as 'internal'!") + (a.name.get, a) + }.toMap } /** From 8117898d585d62e536c7c87f56bc545118cc97b1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Jan 2016 18:42:25 -0800 Subject: [PATCH 13/65] Fix "harmless" exception in peak memory tests The exception was harmless because it didn't actually fail the test. However, the test harness was actually badly written. We used to always assume that the first job will have an ID of 0, but there could very well be other tests sharing the same SparkContext. This is now fixed and we no longer see the exception. As of this commit, all known test failures have been fixed. I'm sure there will be more... --- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 1 - .../org/apache/spark/executor/TaskMetrics.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 16 ++++++++++++---- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index d49e2b62b55b..3f4402bd3a65 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -443,7 +443,6 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th @Override public Option stop(boolean success) { try { - // TODO: this will throw NPE taskContext.taskMetrics().incPeakExecutionMemory(getPeakMemoryUsedBytes()); if (stopping) { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 9e646a4b6165..9b4a9a659dcd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -50,7 +50,7 @@ class TaskMetrics( this(InternalAccumulator.create().map { a => (a.name.get, a) }.toMap, host) } - // Needed for Java + // Needed for Java tests def this() { this(TaskMetrics.getCachedHostName) } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index eac3fe9fbb4e..19487e20e50d 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -328,9 +328,16 @@ private[spark] object AccumulatorSuite { testName: String)(testBody: => Unit): Unit = { val listener = new SaveInfoListener sc.addSparkListener(listener) + + // Find next job ID + var nextJobId = -1 + listener.registerJobCompletionCallback { jobId => nextJobId = jobId + 1 } + sc.parallelize(1 to 10).count() + require(nextJobId > 0, "bad test: job ID was not updated even after a count()") + // Register asserts in job completion callback to avoid flakiness listener.registerJobCompletionCallback { jobId => - if (jobId == 0) { + if (jobId == nextJobId) { // The first job is a dummy one to verify that the accumulator does not already exist val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) assert(!accums.exists(_.name == PEAK_EXECUTION_MEMORY)) @@ -340,9 +347,9 @@ private[spark] object AccumulatorSuite { .flatMap(_.accumulables.values) .find(_.name == PEAK_EXECUTION_MEMORY) .getOrElse { - throw new TestFailedException( - s"peak execution memory accumulator not set in '$testName'", 0) - } + throw new TestFailedException( + s"peak execution memory accumulator not set in '$testName'", 0) + } assert(accum.value.toLong > 0) } } @@ -370,6 +377,7 @@ private class SaveInfoListener extends SparkListener { override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { if (jobCompletionCallback != null) { + // TODO: exceptions thrown here do not actually fail the test! jobCompletionCallback(jobEnd.jobId) } } From 6bd9c0a1caaac38be75f4c8269225a53c75d6509 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Jan 2016 11:13:07 -0800 Subject: [PATCH 14/65] Simplify internal accumulator update mechanism Instead of passing in a callback, we can just return the accumulator values directly, which we have. "We" here refers to TaskMetrics. --- .../scala/org/apache/spark/TaskContext.scala | 5 - .../org/apache/spark/TaskContextImpl.scala | 49 +++--- .../org/apache/spark/executor/Executor.scala | 3 - .../apache/spark/executor/TaskMetrics.scala | 140 ++++++++++-------- .../org/apache/spark/scheduler/Task.scala | 1 - .../sql/execution/ui/SQLListenerSuite.scala | 9 +- 6 files changed, 106 insertions(+), 101 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index fcd2093607c3..6411800e68fb 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -186,9 +186,4 @@ abstract class TaskContext extends Serializable { * accumulator id and the value of the Map is the latest accumulator local value. */ private[spark] def collectAccumulators(): Map[Long, Any] - - /** - * Accumulators for tracking internal metrics indexed by the name. - */ - private[spark] val internalMetricsToAccumulators: Map[String, Accumulator[Long]] } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index e5e67e7b6f2c..0b3491a4e2af 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -17,7 +17,8 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager @@ -37,6 +38,11 @@ private[spark] class TaskContextImpl( extends TaskContext with Logging { + /** + * Metrics associated with this task. + */ + override val taskMetrics: TaskMetrics = new TaskMetrics(internalAccumulators) + // For backwards-compatibility; this method is now deprecated as of 1.3.0. override def attemptId(): Long = taskAttemptId @@ -101,9 +107,19 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - @transient private val accumulators = new HashMap[Long, Accumulable[_, _]] + /** + * All accumulators used in this task indexed by accumulator ID. + */ + @transient private val accumulators = new mutable.HashMap[Long, Accumulable[_, _]] + + // Register the initial set of internal accumulators. + // Future ones will be registered through `registerAccumulator`. + internalAccumulators.foreach { a => accumulators(a.id) = a } private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = synchronized { + if (a.isInternal) { + taskMetrics.registerInternalAccum(a) + } accumulators(a.id) = a } @@ -114,33 +130,4 @@ private[spark] class TaskContextImpl( private[spark] override def collectAccumulators(): Map[Long, Any] = synchronized { accumulators.mapValues(_.localValue).toMap } - - private[spark] override val internalMetricsToAccumulators: Map[String, Accumulator[Long]] = { - // Explicitly register internal accumulators here because these are not captured in the - // task closure and are already deserialized. Additionally, since the driver expects - // us to update these accumulators and report back partial values, we should reset - // each accumulator to zero. Otherwise, we may end up double counting in local mode. - internalAccumulators.map { a => - registerAccumulator(a) - a.resetValue() - assert(a.name.isDefined, "internal accumulator is expected to be named") - assert(a.isInternal, "internal accumulator is not marked as 'internal'!") - (a.name.get, a) - }.toMap - } - - /** - * Metrics associated with this task. - */ - val taskMetrics: TaskMetrics = { - val testing = sys.props.contains("spark.testing") - if (testing && internalMetricsToAccumulators.isEmpty) { - // In tests, we may construct our own dummy TaskContexts where the list of internal - // accumulators is empty. Since TaskMetrics complains if it doesn't find the expected - // accumulators, we just pass in a list of dummy ones here. - new TaskMetrics(InternalAccumulator.create().map { a => (a.name.get, a) }.toMap) - } else { - new TaskMetrics(internalMetricsToAccumulators) - } - } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 425b8370ae9b..34362e8d8224 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -249,7 +249,6 @@ private[spark] class Executor( m.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) m.setJvmGCTime(computeTotalGcTime() - startGCTime) m.setResultSerializationTime(afterSerialization - beforeSerialization) - m.updateAccumulators() } val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) @@ -301,7 +300,6 @@ private[spark] class Executor( task.metrics.map { m => m.setExecutorRunTime(System.currentTimeMillis() - taskStart) m.setJvmGCTime(computeTotalGcTime() - startGCTime) - m.updateAccumulators() m } } @@ -427,7 +425,6 @@ private[spark] class Executor( taskRunner.task.metrics.foreach { metrics => metrics.mergeShuffleReadMetrics() metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - metrics.updateAccumulators() if (isLocal) { // JobProgressListener will hold an reference of it during diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 9b4a9a659dcd..8378e855cc5e 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -19,9 +19,10 @@ package org.apache.spark.executor import java.util.concurrent.ConcurrentHashMap +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Accumulator, InternalAccumulator} +import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils @@ -39,15 +40,15 @@ import org.apache.spark.util.Utils * shipping off at any time to consumers of the SparkListener interface. */ @deprecated("TaskMetrics will be made private in a future version.", "2.0.0") -class TaskMetrics( - accumMap: Map[String, Accumulator[Long]], +class TaskMetrics private[spark] ( + initialAccums: Seq[Accumulator[Long]], val hostname: String = TaskMetrics.getCachedHostName) extends Serializable { import InternalAccumulator._ def this(host: String) { - this(InternalAccumulator.create().map { a => (a.name.get, a) }.toMap, host) + this(InternalAccumulator.create()) } // Needed for Java tests @@ -55,23 +56,22 @@ class TaskMetrics( this(TaskMetrics.getCachedHostName) } - // Each metric is internally represented as an accumulator - private val _executorDeserializeTime: Accumulator[Long] = getAccum(EXECUTOR_DESERIALIZE_TIME) - private val _executorRunTime: Accumulator[Long] = getAccum(EXECUTOR_RUN_TIME) - private val _resultSize: Accumulator[Long] = getAccum(RESULT_SIZE) - private val _jvmGCTime: Accumulator[Long] = getAccum(JVM_GC_TIME) - private val _resultSerializationTime: Accumulator[Long] = getAccum(RESULT_SERIALIZATION_TIME) - private val _memoryBytesSpilled: Accumulator[Long] = getAccum(MEMORY_BYTES_SPILLED) - private val _diskBytesSpilled: Accumulator[Long] = getAccum(DISK_BYTES_SPILLED) - private val _peakExecutionMemory: Accumulator[Long] = getAccum(PEAK_EXECUTION_MEMORY) - /** - * Return the internal accumulator associated with the specified metric, assuming it exists. + * Mapping from metric name to the corresponding internal accumulator. */ - private def getAccum(name: String): Accumulator[Long] = { - assert(accumMap.contains(name), s"metric '$name' is missing") - accumMap(name) - } + private val accumMap = new mutable.HashMap[String, Accumulable[_, _]] + + initialAccums.foreach(registerInternalAccum) + + // Each metric is internally represented as an accumulator + private val _executorDeserializeTime = getLongAccum(EXECUTOR_DESERIALIZE_TIME) + private val _executorRunTime = getLongAccum(EXECUTOR_RUN_TIME) + private val _resultSize = getLongAccum(RESULT_SIZE) + private val _jvmGCTime = getLongAccum(JVM_GC_TIME) + private val _resultSerializationTime = getLongAccum(RESULT_SERIALIZATION_TIME) + private val _memoryBytesSpilled = getLongAccum(MEMORY_BYTES_SPILLED) + private val _diskBytesSpilled = getLongAccum(DISK_BYTES_SPILLED) + private val _peakExecutionMemory = getLongAccum(PEAK_EXECUTION_MEMORY) /** * Time taken on the executor to deserialize this task. @@ -125,6 +125,24 @@ class TaskMetrics( private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + /** + * Register an internal accumulator as a new metric. + */ + private[spark] def registerInternalAccum(a: Accumulable[_, _]): Unit = { + assert(a.name.isDefined, s"internal accumulator (${a.id}) is expected to have a name") + val name = a.name.get + assert(a.isInternal, s"internal accumulator $name (${a.id}) is not marked as 'internal'") + assert(!accumMap.contains(name), s"found duplicate internal accumulator name: $name") + accumMap(name) = a + } + + /** + * Return a Long accumulator associated with the specified metric, assuming it exists. + */ + private def getLongAccum(name: String): Accumulator[Long] = { + TaskMetrics.getLongAccum(accumMap, name) + } + /* ============================ * | OUTPUT METRICS | @@ -143,7 +161,7 @@ class TaskMetrics( */ def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = synchronized { _outputMetrics.getOrElse { - val metrics = new OutputMetrics(writeMethod, accumMap) + val metrics = new OutputMetrics(writeMethod, accumMap.toMap) _outputMetrics = Some(metrics) metrics } @@ -168,7 +186,7 @@ class TaskMetrics( private[spark] def registerInputMetrics(readMethod: DataReadMethod): InputMetrics = { synchronized { val metrics = _inputMetrics.getOrElse { - val metrics = new InputMetrics(readMethod, accumMap) + val metrics = new InputMetrics(readMethod, accumMap.toMap) _inputMetrics = Some(metrics) metrics } @@ -202,7 +220,7 @@ class TaskMetrics( */ def registerShuffleWriteMetrics(): ShuffleWriteMetrics = synchronized { _shuffleWriteMetrics.getOrElse { - val metrics = new ShuffleWriteMetrics(accumMap) + val metrics = new ShuffleWriteMetrics(accumMap.toMap) _shuffleWriteMetrics = Some(metrics) metrics } @@ -248,7 +266,7 @@ class TaskMetrics( */ private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { val agg = _shuffleReadMetrics.getOrElse { - val metrics = new ShuffleReadMetrics(accumMap) + val metrics = new ShuffleReadMetrics(accumMap.toMap) _shuffleReadMetrics = Some(metrics) metrics } @@ -261,30 +279,19 @@ class TaskMetrics( } - /* ================================== * - | OTHER THINGS... WIP | - * ================================== */ + /* =========================== * + | OTHER THINGS | + * =========================== */ /** - * Storage statuses of any blocks that have been updated as a result of this task. + * Return a map from accumulator ID to the accumulator's latest value in this task. */ - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - - private var _accumulatorUpdates: Map[Long, Any] = Map.empty - @transient private var _accumulatorsUpdater: () => Map[Long, Any] = null - - private[spark] def updateAccumulators(): Unit = synchronized { - _accumulatorUpdates = _accumulatorsUpdater() - } + def accumulatorUpdates(): Map[Long, Any] = accumMap.values.map { a => (a.id, a.value) }.toMap /** - * Return the latest updates of accumulators in this task. + * Storage statuses of any blocks that have been updated as a result of this task. */ - def accumulatorUpdates(): Map[Long, Any] = _accumulatorUpdates - - private[spark] def setAccumulatorsUpdater(accumulatorsUpdater: () => Map[Long, Any]): Unit = { - _accumulatorsUpdater = accumulatorsUpdater - } + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } @@ -321,11 +328,11 @@ class InputMetrics private ( private[executor] def this( readMethod: DataReadMethod.Value, - accumMap: Map[String, Accumulator[Long]]) { + accumMap: Map[String, Accumulable[_, _]]) { this( readMethod, - accumMap(InternalAccumulator.input.BYTES_READ), - accumMap(InternalAccumulator.input.RECORDS_READ)) + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.RECORDS_READ)) } /** @@ -370,11 +377,11 @@ class OutputMetrics private ( private[executor] def this( writeMethod: DataWriteMethod.Value, - accumMap: Map[String, Accumulator[Long]]) { + accumMap: Map[String, Accumulable[_, _]]) { this( writeMethod, - accumMap(InternalAccumulator.output.BYTES_WRITTEN), - accumMap(InternalAccumulator.output.RECORDS_WRITTEN)) + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.BYTES_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.RECORDS_WRITTEN)) } /** @@ -405,14 +412,14 @@ class ShuffleReadMetrics private ( _recordsRead: Accumulator[Long]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulator[Long]]) { + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - accumMap(InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), - accumMap(InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), - accumMap(InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), - accumMap(InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), - accumMap(InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), - accumMap(InternalAccumulator.shuffleRead.RECORDS_READ)) + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) } /** @@ -496,11 +503,11 @@ class ShuffleWriteMetrics private ( _shuffleWriteTime: Accumulator[Long]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulator[Long]]) { + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - accumMap(InternalAccumulator.shuffleWrite.BYTES_WRITTEN), - accumMap(InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), - accumMap(InternalAccumulator.shuffleWrite.WRITE_TIME)) + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) } /** @@ -559,4 +566,21 @@ private[spark] object TaskMetrics { val canonicalHost = hostNameCache.putIfAbsent(host, host) if (canonicalHost != null) canonicalHost else host } + + /** + * Return a Long accumulator associated with the specified metric, assuming it exists. + */ + def getLongAccum( + accumMap: scala.collection.Map[String, Accumulable[_, _]], + name: String): Accumulator[Long] = { + assert(accumMap.contains(name), s"metric '$name' is missing") + try { + // Note: we can't do pattern matching here because types are erased by compile time + accumMap(name).asInstanceOf[Accumulator[Long]] + } catch { + case _: ClassCastException => + throw new SparkException(s"attempted to access invalid accumulator $name as a long metric") + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e1139958997d..16d46d6464f8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -78,7 +78,6 @@ private[spark] abstract class Task[T]( internalAccumulators, runningLocally = false) TaskContext.setTaskContext(context) - context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 11a6ce91116f..5f31e10060a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.ui import java.util.Properties +import org.mockito.Mockito.{mock, when} + import org.apache.spark.{SparkException, SparkContext, SparkConf, SparkFunSuite} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -27,6 +29,7 @@ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.LongSQLMetricValue import org.apache.spark.sql.test.SharedSQLContext + class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ @@ -67,9 +70,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { ) private def createTaskMetrics(accumulatorUpdates: Map[Long, Long]): TaskMetrics = { - val metrics = new TaskMetrics - metrics.setAccumulatorsUpdater(() => accumulatorUpdates.mapValues(new LongSQLMetricValue(_))) - metrics.updateAccumulators() + val metrics = mock(classOf[TaskMetrics]) + when(metrics.accumulatorUpdates()) + .thenReturn(accumulatorUpdates.mapValues(new LongSQLMetricValue(_))) metrics } From ed293282118c414e6d5c405a9c2222616ce9d3bd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Jan 2016 13:43:24 -0800 Subject: [PATCH 15/65] Fix tests --- .../scala/org/apache/spark/TaskContext.scala | 13 ++--- .../org/apache/spark/TaskContextImpl.scala | 33 ++++++----- .../apache/spark/executor/TaskMetrics.scala | 2 + .../org/apache/spark/scheduler/Task.scala | 2 +- .../sort/UnsafeShuffleWriterSuite.java | 2 - .../org/apache/spark/AccumulatorSuite.scala | 57 +++++++++---------- 6 files changed, 55 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 6411800e68fb..a29ffa1e8168 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -170,20 +170,19 @@ abstract class TaskContext extends Serializable { private[spark] def taskMemoryManager(): TaskMemoryManager /** - * Register an accumulator that belongs to this task. Accumulators must call this method when - * deserializing in executors. + * All accumulators used in this task. */ - private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit + private[spark] def accumulators: Seq[Accumulable[_, _]] /** - * Return the local values of internal accumulators that belong to this task. The key of the Map - * is the accumulator id and the value of the Map is the latest accumulator local value. + * Register an accumulator that belongs to this task. Accumulators must call this method when + * deserializing in executors. */ - private[spark] def collectInternalAccumulators(): Map[Long, Any] + private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit /** * Return the local values of accumulators that belong to this task. The key of the Map is the * accumulator id and the value of the Map is the latest accumulator local value. */ - private[spark] def collectAccumulators(): Map[Long, Any] + private[spark] def collectAccumulatorValues(): Map[Long, Any] } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 0b3491a4e2af..71e11d80c770 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -43,8 +43,17 @@ private[spark] class TaskContextImpl( */ override val taskMetrics: TaskMetrics = new TaskMetrics(internalAccumulators) + /** + * All accumulators used in this task indexed by accumulator ID. + */ + @transient private val _accumulators = new mutable.HashMap[Long, Accumulable[_, _]] + + // Register the initial set of internal accumulators. + // Future ones will be registered through `registerAccumulator`. + internalAccumulators.foreach { a => _accumulators(a.id) = a } + // For backwards-compatibility; this method is now deprecated as of 1.3.0. - override def attemptId(): Long = taskAttemptId + override val attemptId: Long = taskAttemptId // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] @@ -107,27 +116,23 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - /** - * All accumulators used in this task indexed by accumulator ID. - */ - @transient private val accumulators = new mutable.HashMap[Long, Accumulable[_, _]] - - // Register the initial set of internal accumulators. - // Future ones will be registered through `registerAccumulator`. - internalAccumulators.foreach { a => accumulators(a.id) = a } + private[spark] def accumulators: Seq[Accumulable[_, _]] = _accumulators.values.toSeq private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = synchronized { if (a.isInternal) { taskMetrics.registerInternalAccum(a) } - accumulators(a.id) = a + _accumulators(a.id) = a } - private[spark] override def collectInternalAccumulators(): Map[Long, Any] = synchronized { - accumulators.filter(_._2.isInternal).mapValues(_.localValue).toMap + private[spark] override def collectAccumulatorValues(): Map[Long, Any] = synchronized { + _accumulators.mapValues(_.localValue).toMap } - private[spark] override def collectAccumulators(): Map[Long, Any] = synchronized { - accumulators.mapValues(_.localValue).toMap + // For testing only. + private[spark] def findTestAccum(): Option[Accumulator[Long]] = { + accumulators + .find(_.name == Some(InternalAccumulator.TEST_ACCUM)) + .map(_.asInstanceOf[Accumulator[Long]]) } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 8378e855cc5e..b02c12fa3c35 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -133,6 +133,8 @@ class TaskMetrics private[spark] ( val name = a.name.get assert(a.isInternal, s"internal accumulator $name (${a.id}) is not marked as 'internal'") assert(!accumMap.contains(name), s"found duplicate internal accumulator name: $name") + // Note: reset the value here so we don't double count the values in local mode + a.resetValue() accumMap(name) = a } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 16d46d6464f8..568aa8e2c590 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -83,7 +83,7 @@ private[spark] abstract class Task[T]( kill(interruptThread = false) } try { - (runTask(context), context.collectAccumulators()) + (runTask(context), context.collectAccumulatorValues()) } finally { context.markTaskCompleted() try { diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 625fdd57eb5d..876c3a228364 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -191,8 +191,6 @@ public Tuple2 answer( }); when(taskContext.taskMetrics()).thenReturn(taskMetrics); - when(taskContext.internalMetricsToAccumulators()).thenReturn(null); - when(shuffleDep.serializer()).thenReturn(Option.apply(serializer)); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 19487e20e50d..8f93e0ddbdd9 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -59,7 +59,8 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex longAcc.value should be (210L + maxInt * 20) } - test ("value not assignable from tasks") { + // TODO: fix me + ignore("value not assignable from tasks") { sc = new SparkContext("local", "test") val acc : Accumulator[Int] = sc.accumulator(0) @@ -84,7 +85,8 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("value not readable in tasks") { + // TODO: fix me + ignore("value not readable in tasks") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -163,17 +165,14 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext("local", "test") val accums = InternalAccumulator.create(sc) val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null, accums) - val internalMetricsToAccums = taskContext.internalMetricsToAccumulators - val collectedInternalAccums = taskContext.collectInternalAccumulators() - val collectedAccums = taskContext.collectAccumulators() - assert(internalMetricsToAccums.size > 0) - assert(internalMetricsToAccums.values.forall(_.isInternal)) - assert(internalMetricsToAccums.contains(TEST_ACCUM)) - val testAccum = internalMetricsToAccums(TEST_ACCUM) - assert(collectedInternalAccums.size === internalMetricsToAccums.size) - assert(collectedInternalAccums.size === collectedAccums.size) - assert(collectedInternalAccums.contains(testAccum.id)) - assert(collectedAccums.contains(testAccum.id)) + val accumulators = taskContext.accumulators + val accumulatorValues = taskContext.collectAccumulatorValues() + assert(accumulators.size > 0) + assert(accumulators.forall(_.isInternal)) + val testAccum = taskContext.findTestAccum() + assert(testAccum.isDefined) + assert(accumulatorValues.size === accumulators.size) + assert(accumulatorValues.contains(testAccum.get.id)) } test("internal accumulators in a stage") { @@ -183,7 +182,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 1 + TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum().get += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -193,11 +192,11 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(stageInfos.size === 1) assert(taskInfos.size === numPartitions) // The accumulator values should be merged in the stage - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUM) + val stageAccum = findTestAccum(stageInfos.head.accumulables.values) assert(stageAccum.value.toLong === numPartitions) // The accumulator should be updated locally on each task val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUM) + val taskAccum = findTestAccum(taskInfo.accumulables) assert(taskAccum.update.isDefined) assert(taskAccum.update.get.toLong === 1) taskAccum.value.toLong @@ -218,17 +217,17 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 1 + TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 10 + TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUM) += 100 + TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -237,9 +236,9 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val stageInfos = listener.getCompletedStageInfos assert(stageInfos.size === 3) val (firstStageAccum, secondStageAccum, thirdStageAccum) = - (findAccumulableInfo(stageInfos(0).accumulables.values, TEST_ACCUM), - findAccumulableInfo(stageInfos(1).accumulables.values, TEST_ACCUM), - findAccumulableInfo(stageInfos(2).accumulables.values, TEST_ACCUM)) + (findTestAccum(stageInfos(0).accumulables.values), + findTestAccum(stageInfos(1).accumulables.values), + findTestAccum(stageInfos(2).accumulables.values)) assert(firstStageAccum.value.toLong === numPartitions) assert(secondStageAccum.value.toLong === numPartitions * 10) assert(thirdStageAccum.value.toLong === numPartitions * 2 * 100) @@ -258,11 +257,9 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex /** * Return the accumulable info that matches the specified name. */ - private def findAccumulableInfo( - accums: Iterable[AccumulableInfo], - name: String): AccumulableInfo = { - accums.find { a => a.name == name }.getOrElse { - throw new TestFailedException(s"internal accumulator '$name' not found", 0) + private def findTestAccum(accums: Iterable[AccumulableInfo]): AccumulableInfo = { + accums.find { a => a.name == TEST_ACCUM }.getOrElse { + fail(s"unable to find internal accumulator called $TEST_ACCUM") } } @@ -278,7 +275,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => val taskContext = TaskContext.get() - taskContext.internalMetricsToAccumulators(TEST_ACCUM) += 1 + taskContext.asInstanceOf[TaskContextImpl].findTestAccum().get += 1 // Fail the first attempts of a subset of the tasks if (failCondition(i) && taskContext.attemptNumber() == 0) { throw new Exception("Failing a task intentionally.") @@ -291,13 +288,13 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val taskInfos = listener.getCompletedTaskInfos assert(stageInfos.size === 1) assert(taskInfos.size === numPartitions + numFailedPartitions) - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUM) + val stageAccum = findTestAccum(stageInfos.head.accumulables.values) // We should not double count values in the merged accumulator assert(stageAccum.value.toLong === numPartitions) val taskAccumValues = taskInfos.flatMap { taskInfo => if (!taskInfo.failed) { // If a task succeeded, its update value should always be 1 - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUM) + val taskAccum = findTestAccum(taskInfo.accumulables) assert(taskAccum.update.isDefined) assert(taskAccum.update.get.toLong === 1) Some(taskAccum.value.toLong) From 20119126a26420b30c4bba417d2c04aaffac305a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Jan 2016 15:19:56 -0800 Subject: [PATCH 16/65] Clean up This commit addresses outstanding TODO's and makes the deprecated APIs DeveloperApi instead. This allows us to deal with how to do the deprecation properly later. This commit also reverts a few unnecessary changes to reduce the size of the diff. --- .../scala/org/apache/spark/Accumulators.scala | 46 ++++++++++++------- .../scala/org/apache/spark/SparkEnv.scala | 3 ++ .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 4 +- .../apache/spark/executor/TaskMetrics.scala | 29 ++++++++---- .../org/apache/spark/rdd/HadoopRDD.scala | 2 - .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 - .../shuffle/sort/SortShuffleWriter.scala | 5 +- .../org/apache/spark/AccumulatorSuite.scala | 36 +++++++-------- .../org/apache/spark/SparkFunSuite.scala | 2 + .../org/apache/spark/ui/StagePageSuite.scala | 4 +- .../datasources/SqlNewHadoopRDD.scala | 2 - .../sql/util/DataFrameCallbackSuite.scala | 4 +- .../spark/tools/StoragePerfTester.scala | 2 +- 14 files changed, 79 insertions(+), 64 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index bcbc1f21f682..c83738281bb4 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.Utils * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are * accumulating a set. You will add items to the set, and you will union two sets together. * - * TODO: document thread-safety. + * Operations are not thread-safe. * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` @@ -71,7 +71,10 @@ class Accumulable[R, T] private[spark] ( @volatile private var value_ : R = initialValue val zero = param.zero(initialValue) - Accumulators.register(this) + // Avoid leaking accumulators on executors + if (isDriver) { + Accumulators.register(this) + } /** * If this [[Accumulable]] is internal. Internal [[Accumulable]]s will be reported to the driver @@ -109,9 +112,14 @@ class Accumulable[R, T] private[spark] ( def merge(term: R): Unit = { value_ = param.addInPlace(value_, term) } /** - * Access the accumulator's current value; only allowed on master. + * Access the accumulator's current value; only allowed on driver. */ - def value: R = value_ + def value: R = { + if (!isDriver) { + throw new UnsupportedOperationException("Can't read accumulator value in task") + } + value_ + } /** * Get the current value of this accumulator from within a task. @@ -121,40 +129,41 @@ class Accumulable[R, T] private[spark] ( * * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. - * - * TODO: probably don't need this. */ def localValue: R = value_ /** - * Set the accumulator's value. + * Set the accumulator's value; only allowed on driver. */ def value_= (newValue: R): Unit = { + if (!isDriver) { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } value_ = newValue } /** * Set the accumulator's value. */ - def setValue(newValue: R): Unit = { - this.value = newValue - } + private[spark] def setValue(newValue: R): Unit = { value_ = newValue } /** * Reset the accumulator's value to zero. */ - private[spark] def resetValue(): Unit = { - setValue(zero) + private[spark] def resetValue(): Unit = { setValue(zero) } + + /** + * Whether we are on the driver or the executors. + * Note: in local mode, this will inevitably return true even if we're on the executor. + */ + private def isDriver: Boolean = { + Option(SparkEnv.get).map(_.isDriver).getOrElse(true) } // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() // Automatically register the accumulator when it is deserialized with the task closure. - // - // Note internal accumulators sent with task are deserialized before the TaskContext is created - // and are registered in the TaskContext constructor. Other internal accumulators, such SQL - // metrics, still need to register here. val taskContext = TaskContext.get() if (taskContext != null) { taskContext.registerAccumulator(this) @@ -319,6 +328,7 @@ private[spark] object Accumulators extends Logging { * This global map holds the original accumulator objects that are created on the driver. * It keeps weak references to these objects so that accumulators can be garbage-collected * once the RDDs and user-code that reference them are cleaned up. + * TODO: Don't use a global map; these should be tied to a SparkContext at the very least. */ val originals = mutable.Map[Long, WeakReference[Accumulable[_, _]]]() @@ -356,6 +366,10 @@ private[spark] object Accumulators extends Logging { } } + def clear(): Unit = synchronized { + originals.clear() + } + } private[spark] object InternalAccumulator { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index b98cc964eda8..b681ddf616b7 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -71,6 +71,9 @@ class SparkEnv ( val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { + // Note: always returns true in local mode + private[spark] def isDriver: Boolean = blockManager.blockManagerId.isDriver + // TODO Remove actorSystem @deprecated("Actor system is no longer supported as of 1.4.0", "1.4.0") val actorSystem: ActorSystem = _actorSystem diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index a29ffa1e8168..182c486dba10 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -153,7 +153,7 @@ abstract class TaskContext extends Serializable { */ def taskAttemptId(): Long - @deprecated("TaskMetrics will be made private in a future version", "2.0.0") + @DeveloperApi def taskMetrics(): TaskMetrics /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 300bc5e7cb91..689b3077b060 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -373,8 +373,8 @@ object SparkHadoopUtil { /** * Number of records to update input metrics when reading from HadoopRDDs. * - * Each update is potentially expensive because we need to use reflection to access the Hadoop - * FileSystem API of interest (it is only available in 2.5), so we should do it sparingly. + * Each update is potentially expensive because we need to use reflection to access the + * Hadoop FileSystem API of interest (only available in 2.5), so we should do this sparingly. */ private[spark] val UPDATE_INPUT_METRICS_INTERVAL_RECORDS = 1000 diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index b02c12fa3c35..254aca4eb6ca 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -23,12 +23,16 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils +// TODO: make everything here private + /** + * :: DeveloperApi :: * Metrics tracked during the execution of a task. * * This class is used to house metrics both for in-progress and completed tasks. In executors, @@ -39,7 +43,7 @@ import org.apache.spark.util.Utils * So, when adding new fields, take into consideration that the whole object can be serialized for * shipping off at any time to consumers of the SparkListener interface. */ -@deprecated("TaskMetrics will be made private in a future version.", "2.0.0") +@DeveloperApi class TaskMetrics private[spark] ( initialAccums: Seq[Accumulator[Long]], val hostname: String = TaskMetrics.getCachedHostName) @@ -293,15 +297,17 @@ class TaskMetrics private[spark] ( /** * Storage statuses of any blocks that have been updated as a result of this task. */ + // TODO: make me an accumulator var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } /** + * :: DeveloperApi :: * Method by which input data was read. Network means that the data was read over the network * from a remote block manager (which may have stored the data on-disk or in-memory). */ -@deprecated("DataReadMethod will be made private in a future version.", "2.0.0") +@DeveloperApi object DataReadMethod extends Enumeration with Serializable { type DataReadMethod = Value val Memory, Disk, Hadoop, Network = Value @@ -309,9 +315,10 @@ object DataReadMethod extends Enumeration with Serializable { /** + * :: DeveloperApi :: * Method by which output data was written. */ -@deprecated("DataWriteMethod will be made private in a future version.", "2.0.0") +@DeveloperApi object DataWriteMethod extends Enumeration with Serializable { type DataWriteMethod = Value val Hadoop = Value @@ -319,9 +326,10 @@ object DataWriteMethod extends Enumeration with Serializable { /** + * :: DeveloperApi :: * Metrics about reading input data. */ -@deprecated("InputMetrics will be made private in a future version.", "2.0.0") +@DeveloperApi class InputMetrics private ( val readMethod: DataReadMethod.Value, _bytesRead: Accumulator[Long], @@ -368,9 +376,10 @@ class InputMetrics private ( /** + * :: DeveloperApi :: * Metrics about writing output data. */ -@deprecated("OutputMetrics will be made private in a future version.", "2.0.0") +@DeveloperApi class OutputMetrics private ( val writeMethod: DataWriteMethod.Value, _bytesWritten: Accumulator[Long], @@ -402,9 +411,10 @@ class OutputMetrics private ( /** + * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. */ -@deprecated("ShuffleReadMetrics will be made private in a future version.", "2.0.0") +@DeveloperApi class ShuffleReadMetrics private ( _remoteBlocksFetched: Accumulator[Long], _localBlocksFetched: Accumulator[Long], @@ -431,7 +441,7 @@ class ShuffleReadMetrics private ( * many places only to merge their values together later. In the future, we should revisit * whether this is needed. * - * * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. + * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. */ private[spark] def this() { this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) @@ -496,9 +506,10 @@ class ShuffleReadMetrics private ( /** + * :: DeveloperApi :: * Metrics pertaining to shuffle data written in a given task. */ -@deprecated("ShuffleWriteMetrics will be made private in a future version.", "2.0.0") +@DeveloperApi class ShuffleWriteMetrics private ( _bytesWritten: Accumulator[Long], _recordsWritten: Accumulator[Long], @@ -540,8 +551,6 @@ class ShuffleWriteMetrics private ( */ def shuffleWriteTime: Long = _shuffleWriteTime.value - // TODO: these are not thread-safe. Is that OK? - private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) private[spark] def incWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 10018c200cf7..fd33cfc7e97f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -220,8 +220,6 @@ class HadoopRDD[K, V]( case _ => SqlNewHadoopRDDState.unsetInputFileName() } - // TODO: duplicate code alert! - // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes val getBytesReadCallback: Option[() => Long] = split.inputSplit.value match { diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ec49ba45eec0..af5949235d45 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -124,8 +124,6 @@ class NewHadoopRDD[K, V]( val inputMetrics = context.taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) - // TODO: duplicate code alert! - // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index b2cf17e67ecb..7eb3d9603736 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -45,6 +45,8 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null + private val writeMetrics = context.taskMetrics().registerShuffleWriteMetrics() + /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { sorter = if (dep.mapSideCombine) { @@ -90,8 +92,7 @@ private[spark] class SortShuffleWriter[K, V, C]( if (sorter != null) { val startTime = System.nanoTime() sorter.stop() - context.taskMetrics.shuffleWriteMetrics.foreach( - _.incWriteTime(System.nanoTime - startTime)) + writeMetrics.incWriteTime(System.nanoTime - startTime) sorter = null } } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 8f93e0ddbdd9..f599c9b27d2d 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -45,7 +45,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("basic accumulation"){ + test("basic accumulation") { sc = new SparkContext("local", "test") val acc : Accumulator[Int] = sc.accumulator(0) @@ -59,16 +59,16 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex longAcc.value should be (210L + maxInt * 20) } - // TODO: fix me - ignore("value not assignable from tasks") { - sc = new SparkContext("local", "test") + test("value not assignable from tasks") { + // Note: this does not work in local mode because we can't tell whether we're on the driver + sc = new SparkContext("local-cluster[1,1,1024]", "test") val acc : Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) an [Exception] should be thrownBy {d.foreach{x => acc.value = x}} } - test ("add value to collection accumulators") { + test("add value to collection accumulators") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -85,23 +85,20 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - // TODO: fix me - ignore("value not readable in tasks") { + test("value not readable in tasks") { val maxI = 1000 - for (nThreads <- List(1, 10)) { // test single & multi-threaded - sc = new SparkContext("local[" + nThreads + "]", "test") - val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val d = sc.parallelize(1 to maxI) - an [SparkException] should be thrownBy { - d.foreach { - x => acc.value += x - } + // Note: this does not work in local mode because we can't tell whether we're on the driver + sc = new SparkContext("local-cluster[1,1,1024]", "test") + val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val d = sc.parallelize(1 to maxI) + an [SparkException] should be thrownBy { + d.foreach { + x => acc.value += x } - resetSparkContext() } } - test ("collection accumulators") { + test("collection accumulators") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded @@ -127,7 +124,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("localValue readable in tasks") { + test("localValue readable in tasks") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -142,7 +139,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("garbage collection") { + test("garbage collection") { // Create an accumulator and let it go out of scope to test that it's properly garbage collected sc = new SparkContext("local", "test") var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) @@ -318,7 +315,6 @@ private[spark] object AccumulatorSuite { /** * Run one or more Spark jobs and verify that the peak execution memory accumulator * is updated afterwards. - * TODO: assert it's also set in task metrics? */ def verifyPeakExecutionMemorySet( sc: SparkContext, diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 9be9db01c7de..d3359c7406e4 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -42,6 +42,8 @@ private[spark] abstract class SparkFunSuite extends FunSuite with Logging { test() } finally { logInfo(s"\n\n===== FINISHED $shortSuiteName: '$testName' =====\n") + // Avoid leaking map entries in tests that use accumulators without SparkContext + Accumulators.clear() } } 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 bc8a4299d9c3..86699e7f5695 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -31,8 +31,6 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener class StagePageSuite extends SparkFunSuite with LocalSparkContext { - import InternalAccumulator._ - test("peak execution memory only displayed if unsafe is enabled") { val unsafeConf = "spark.sql.unsafe.enabled" val conf = new SparkConf(false).set(unsafeConf, "true") @@ -82,7 +80,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) val peakExecutionMemory = 10 - taskInfo.accumulables += new AccumulableInfo(0, PEAK_EXECUTION_MEMORY, + taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY, Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index 8918597df531..2f2e90bca9a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala @@ -134,8 +134,6 @@ private[spark] class SqlNewHadoopRDD[V: ClassTag]( case _ => SqlNewHadoopRDDState.unsetInputFileName() } - // TODO: duplicate code alert! - // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index d1cfa9bf6736..b46b0d2f6040 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -116,8 +116,6 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { // A easy fix is to create a new SQLMetric(including new MetricValue, MetricParam, etc.), but we // can do it later because the impact is just too small (1048576 tasks for 1 MB). ignore("get size metrics by callback") { - import InternalAccumulator._ - val metrics = ArrayBuffer.empty[Long] val listener = new QueryExecutionListener { // Only test successful case here, so no need to implement `onFailure` @@ -139,7 +137,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { def getPeakExecutionMemory(stageId: Int): Long = { val peakMemoryAccumulator = sparkListener.getCompletedStageInfos(stageId).accumulables - .filter(_._2.name == PEAK_EXECUTION_MEMORY) + .filter(_._2.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) assert(peakMemoryAccumulator.size == 1) peakMemoryAccumulator.head._2.value.toLong diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 2c63a870a721..8a5c7c0e730e 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -61,7 +61,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf), new ShuffleWriteMetrics) + new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeKey, writeValue) From 9222f11d2b838b3bd61d62396719e44cb728bf66 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Jan 2016 19:55:50 -0800 Subject: [PATCH 17/65] Do not send TaskMetrics from executors to driver Instead, send only accumulator updates. As of this commit TaskMetrics is only used as a syntactic sugar on the executor side to modify accumulator values by names. Now we no longer send the same thing in two different codepaths. Now that we never send TaskMetrics from executors to the driver, we also never send accumulators that way. Then we can revert some of the accumulator changes. --- .../org/apache/spark/TaskEndReason.scala | 19 +- .../org/apache/spark/executor/Executor.scala | 23 ++- .../apache/spark/executor/TaskMetrics.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 39 ++++- .../spark/scheduler/DAGSchedulerEvent.scala | 3 +- .../org/apache/spark/scheduler/Task.scala | 15 +- .../apache/spark/scheduler/TaskResult.scala | 15 +- .../spark/scheduler/TaskResultGetter.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 5 +- .../spark/scheduler/TaskSetManager.scala | 46 ++++- .../org/apache/spark/util/JsonProtocol.scala | 8 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 164 +++++++----------- .../org/apache/spark/scheduler/FakeTask.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 27 +-- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 9 +- 17 files changed, 210 insertions(+), 176 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 13241b77bf97..d7d0a0611b4e 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -115,8 +115,10 @@ case class ExceptionFailure( description: String, stackTrace: Array[StackTraceElement], fullStackTrace: String, - metrics: Option[TaskMetrics], - private val exceptionWrapper: Option[ThrowableSerializationWrapper]) + exceptionWrapper: Option[ThrowableSerializationWrapper], + accumulatorUpdates: Map[Long, Any] = Map[Long, Any](), + // always None, kept here for backward compatibility + metrics: Option[TaskMetrics] = None) extends TaskFailedReason { /** @@ -124,13 +126,16 @@ case class ExceptionFailure( * driver. This may be set to `false` in the event that the exception is not in fact * serializable. */ - private[spark] def this(e: Throwable, metrics: Option[TaskMetrics], preserveCause: Boolean) { - this(e.getClass.getName, e.getMessage, e.getStackTrace, Utils.exceptionString(e), metrics, - if (preserveCause) Some(new ThrowableSerializationWrapper(e)) else None) + private[spark] def this( + e: Throwable, + accumulatorUpdates: Map[Long, Any], + preserveCause: Boolean) { + this(e.getClass.getName, e.getMessage, e.getStackTrace, Utils.exceptionString(e), + if (preserveCause) Some(new ThrowableSerializationWrapper(e)) else None, accumulatorUpdates) } - private[spark] def this(e: Throwable, metrics: Option[TaskMetrics]) { - this(e, metrics, preserveCause = true) + private[spark] def this(e: Throwable, accumulatorUpdates: Map[Long, Any]) { + this(e, accumulatorUpdates, preserveCause = true) } def exception: Option[Throwable] = exceptionWrapper.flatMap { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 34362e8d8224..4546b5426209 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -251,7 +251,7 @@ private[spark] class Executor( m.setResultSerializationTime(afterSerialization - beforeSerialization) } - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) + val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit @@ -296,20 +296,25 @@ private[spark] class Executor( // the default uncaught exception handler, which will terminate the Executor. logError(s"Exception in $taskName (TID $taskId)", t) - val metrics: Option[TaskMetrics] = Option(task).flatMap { task => - task.metrics.map { m => - m.setExecutorRunTime(System.currentTimeMillis() - taskStart) - m.setJvmGCTime(computeTotalGcTime() - startGCTime) - m + // Collect latest accumulator values to report back to the driver + val accumulatorUpdates: Map[Long, Any] = + if (task != null) { + task.metrics.foreach { m => + m.setExecutorRunTime(System.currentTimeMillis() - taskStart) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) + } + task.collectAccumulatorUpdates() + } else { + Map[Long, Any]() } - } + val serializedTaskEndReason = { try { - ser.serialize(new ExceptionFailure(t, metrics)) + ser.serialize(new ExceptionFailure(t, accumulatorUpdates)) } catch { case _: NotSerializableException => // t is not serializable so just send the stacktrace - ser.serialize(new ExceptionFailure(t, metrics, false)) + ser.serialize(new ExceptionFailure(t, accumulatorUpdates, false)) } } execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 254aca4eb6ca..7e3559e5c1f7 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -52,7 +52,7 @@ class TaskMetrics private[spark] ( import InternalAccumulator._ def this(host: String) { - this(InternalAccumulator.create()) + this(InternalAccumulator.create(), host) } // Needed for Java tests diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 92438ba892cc..4391587e0e54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -209,10 +209,8 @@ class DAGScheduler( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics): Unit = { - eventProcessLoop.post( - CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) + taskInfo: TaskInfo): Unit = { + eventProcessLoop.post(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } /** @@ -1110,6 +1108,29 @@ class DAGScheduler( } } + /** + * Reconstruct [[TaskMetrics]] from accumulator updates. + * + * This is needed for posting task end events to listeners. If the task has failed, + * `accumUpdates` may be null, in which case we just return null. + */ + private def reconstructTaskMetrics(task: Task[_], accumUpdates: Map[Long, Any]): TaskMetrics = { + if (accumUpdates != null) { + task.internalAccumulators.foreach { a => + assert(a.name.isDefined, s"internal accumulator ${a.id} is expected to have a name.") + assert(accumUpdates.contains(a.id), + s"missing entry in task accumulator updates: ${a.name.get} (${a.id})") + val stringValue = accumUpdates(a.id).toString + assert(stringValue.forall(_.isDigit), + s"existing value for accumulator ${a.name.get} (${a.id}) was not a number: $stringValue ") + a.setValue(stringValue.toLong) + } + new TaskMetrics(task.internalAccumulators) + } else { + null + } + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -1119,6 +1140,10 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) + // Executors only send accumulator updates back to the driver, not TaskMetrics. However, + // we still need TaskMetrics to post task end events to listeners, so reconstruct them here. + val taskMetrics = reconstructTaskMetrics(task, event.accumUpdates) + outputCommitCoordinator.taskCompleted( stageId, task.partitionId, @@ -1130,7 +1155,7 @@ class DAGScheduler( if (event.reason != Success) { val attemptId = task.stageAttemptId listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason, - event.taskInfo, event.taskMetrics)) + event.taskInfo, taskMetrics)) } if (!stageIdToStage.contains(task.stageId)) { @@ -1142,7 +1167,7 @@ class DAGScheduler( event.reason match { case Success => listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, - event.reason, event.taskInfo, event.taskMetrics)) + event.reason, event.taskInfo, taskMetrics)) stage.pendingPartitions -= task.partitionId task match { case rt: ResultTask[_, _] => @@ -1637,7 +1662,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case GettingResultEvent(taskInfo) => dagScheduler.handleGetTaskResult(taskInfo) - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + case completion: CompletionEvent => dagScheduler.handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason, exception) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index dda3b6cc7f96..e85516aca493 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -74,8 +74,7 @@ private[scheduler] case class CompletionEvent( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) + taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 53af34a35656..be02f179bef0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -47,7 +47,7 @@ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - internalAccumulators: Seq[Accumulator[Long]]) extends Serializable { + val internalAccumulators: Seq[Accumulator[Long]]) extends Serializable { /** * The key of the Map is the accumulator id and the value of the Map is the latest accumulator @@ -82,7 +82,7 @@ private[spark] abstract class Task[T]( kill(interruptThread = false) } try { - (runTask(context), context.collectAccumulatorValues()) + (runTask(context), collectAccumulatorUpdates()) } finally { context.markTaskCompleted() try { @@ -129,6 +129,17 @@ private[spark] abstract class Task[T]( protected var _executorDeserializeTime: Long = 0 + /** + * Collect the latest values of accumulators used in this task. + */ + def collectAccumulatorUpdates(): Map[Long, Any] = { + if (context != null) { + context.collectAccumulatorValues() + } else { + Map[Long, Any]() + } + } + /** * Whether the task has been killed. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b82c7f3fa54f..bc506cf2e768 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -24,7 +24,6 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkEnv -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockId import org.apache.spark.util.Utils @@ -36,19 +35,18 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) extends TaskResult[T] with Serializable /** A TaskResult that contains the task's return value and accumulator updates. */ -private[spark] -class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], - var metrics: TaskMetrics) +private[spark] class DirectTaskResult[T]( + var valueBytes: ByteBuffer, + var accumUpdates: Map[Long, Any]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false private var valueObject: T = _ - def this() = this(null.asInstanceOf[ByteBuffer], null, null) + def this() = this(null.asInstanceOf[ByteBuffer], null) override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - - out.writeInt(valueBytes.remaining); + out.writeInt(valueBytes.remaining) Utils.writeByteBuffer(valueBytes, out) out.writeInt(accumUpdates.size) @@ -56,11 +54,9 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long out.writeLong(key) out.writeObject(value) } - out.writeObject(metrics) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val blen = in.readInt() val byteVal = new Array[Byte](blen) in.readFully(byteVal) @@ -76,7 +72,6 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long } accumUpdates = _accumUpdates } - metrics = in.readObject().asInstanceOf[TaskMetrics] valueObjectDeserialized = false } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index f4965994d827..e9a745d18fc5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -82,8 +82,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul (deserializedResult, size) } - result.metrics.setResultSize(size) - scheduler.handleSuccessfulTask(taskSetManager, tid, result) + scheduler.handleSuccessfulTask(taskSetManager, tid, result, size.toLong) } catch { case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6e3ef0e54f0f..a8a4c58cc34d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -400,8 +400,9 @@ private[spark] class TaskSchedulerImpl( def handleSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, - taskResult: DirectTaskResult[_]): Unit = synchronized { - taskSetManager.handleSuccessfulTask(tid, taskResult) + taskResult: DirectTaskResult[_], + resultSize: Long): Unit = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult, resultSize) } def handleFailedTask( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index aa39b59d8cce..e3abc12236d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -29,7 +29,6 @@ import scala.math.{max, min} import scala.util.control.NonFatal import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -610,19 +609,22 @@ private[spark] class TaskSetManager( /** * Marks a task as successful and notifies the DAGScheduler that the task has ended. */ - def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = { + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_], resultSize: Long): Unit = { val info = taskInfos(tid) val index = info.index info.markSuccessful() removeRunningTask(tid) + + // Update result size metric in `result`, which is not yet set. + updateResultSize(tid, result, resultSize) + // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not // "deserialize" the value when holding a lock to avoid blocking other threads. So we call // "result.value()" in "TaskResultGetter.enqueueSuccessfulTask" before reaching here. // Note: "result.value()" only deserializes the value when it's called at the first time, so // here "result.value()" just returns the value and won't block other threads. - sched.dagScheduler.taskEnded( - tasks(index), Success, result.value(), result.accumUpdates, info, result.metrics) + sched.dagScheduler.taskEnded(tasks(index), Success, result.value(), result.accumUpdates, info) if (!successful(index)) { tasksSuccessful += 1 logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( @@ -640,6 +642,28 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } + /** + * Set the task result size in the accumulator updates received from the executors. + * + * Note: If we did this on the executors we would have to serialize the result again after + * updating the size, which is potentially expensive. Also, we would have to do something + * extra for indirect task results. It's better to just do all of this in one place on the + * driver. + */ + private def updateResultSize(tid: Long, result: DirectTaskResult[_], resultSize: Long): Unit = { + val index = taskInfos(tid).index + val task = tasks(index) + val accumName = InternalAccumulator.RESULT_SIZE + val resultSizeAccum = task.internalAccumulators.find { a => a.name == Some(accumName) } + assert(resultSizeAccum.isDefined, s"did not find accumulator called '$accumName' in task") + val resultSizeAccumId = resultSizeAccum.get.id + assert(result.accumUpdates.contains(resultSizeAccumId), + s"did not find accumulator called '$accumName' in task result") + assert(result.accumUpdates(resultSizeAccumId) == 0L, + s"task result size '$accumName' should not have been set on the executors") + result.accumUpdates = result.accumUpdates + ((resultSizeAccumId, resultSize)) + } + /** * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the * DAG Scheduler. @@ -653,8 +677,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var taskMetrics : TaskMetrics = null - + var accumUpdates: Map[Long, Any] = Map[Long, Any]() val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { @@ -669,7 +692,12 @@ private[spark] class TaskSetManager( None case ef: ExceptionFailure => - taskMetrics = ef.metrics.orNull + + // ExceptionFailure's might have accumulator updates + if (ef.accumulatorUpdates != null) { + accumUpdates = ef.accumulatorUpdates + } + if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" @@ -721,7 +749,7 @@ private[spark] class TaskSetManager( // always add to failed executors failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). put(info.executorId, clock.getTimeMillis()) - sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics) + sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) addPendingTask(index) if (!isZombie && state != TaskState.KILLED && reason.isInstanceOf[TaskFailedReason] @@ -793,7 +821,7 @@ private[spark] class TaskSetManager( addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null) + sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info) } } } 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 17e401b2178c..7d34106c50cd 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -353,13 +353,12 @@ private[spark] object JsonProtocol { ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => + // TODO: serialize accumulator updates as well? val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) - val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) ("Class Name" -> exceptionFailure.className) ~ ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ - ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ - ("Metrics" -> metrics) + ("Full Stack Trace" -> exceptionFailure.fullStackTrace) case taskCommitDenied: TaskCommitDenied => ("Job ID" -> taskCommitDenied.jobID) ~ ("Partition ID" -> taskCommitDenied.partitionID) ~ @@ -794,7 +793,8 @@ private[spark] object JsonProtocol { val fullStackTrace = Utils.jsonOption(json \ "Full Stack Trace"). map(_.extract[String]).orNull val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) - ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) + ExceptionFailure( + className, description, stackTrace, fullStackTrace, None, metrics = metrics) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled case `taskCommitDenied` => diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 4e678fbac6a3..80a1de6065b4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -801,7 +801,7 @@ class ExecutorAllocationManagerSuite assert(maxNumExecutorsNeeded(manager) === 1) // If the task is failed, we expect it to be resubmitted later. - val taskEndReason = ExceptionFailure(null, null, null, null, null, None) + val taskEndReason = ExceptionFailure(null, null, null, null, None) sc.listenerBus.postToAll(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) assert(maxNumExecutorsNeeded(manager) === 1) } 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 370a284d2950..74975b7fe51e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -23,7 +23,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.language.reflectiveCalls import scala.util.control.NonFatal -import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ @@ -96,8 +95,7 @@ class MyRDD( class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite - extends SparkFunSuite with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -189,7 +187,8 @@ class DAGSchedulerSuite override def jobFailed(exception: Exception): Unit = { failure = exception } } - before { + override def beforeEach(): Unit = { + super.beforeEach() sc = new SparkContext("local", "DAGSchedulerSuite") sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() @@ -211,8 +210,12 @@ class DAGSchedulerSuite dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) } - after { - scheduler.stop() + override def afterEach(): Unit = { + try { + scheduler.stop() + } finally { + super.afterEach() + } } override def afterAll() { @@ -249,19 +252,20 @@ class DAGSchedulerSuite assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent( - taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(taskSet.tasks(i), result._1, result._2)) } } } - private def completeWithAccumulator(accumId: Long, taskSet: TaskSet, - results: Seq[(TaskEndReason, Any)]) { + private def completeWithAccumulator( + accumId: Long, + taskSet: TaskSet, + results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, - Map[Long, Any]((accumId, 1)), createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent( + taskSet.tasks(i), result._1, result._2, Map[Long, Any](accumId -> 1))) } } } @@ -829,23 +833,17 @@ class DAGSchedulerSuite HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // The SparkListener should not receive redundant failure events. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) @@ -882,12 +880,9 @@ class DAGSchedulerSuite HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -900,12 +895,9 @@ class DAGSchedulerSuite assert(countSubmittedMapStageAttempts() === 2) // The second ResultTask fails, with a fetch failure for the output from the second mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(1), FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -952,12 +944,9 @@ class DAGSchedulerSuite assert(countSubmittedReduceStageAttempts() === 1) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -971,12 +960,9 @@ class DAGSchedulerSuite assert(countSubmittedReduceStageAttempts() === 2) // A late FetchFailed arrives from the second task in the original reduce stage. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(1), FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -1007,45 +993,33 @@ class DAGSchedulerSuite assert(shuffleStage.numAvailableOutputs === 0) // should be ignored for being too old - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 0) // should work because it's a non-failed host (so the available map outputs will increase) - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostB", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostB", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 1) // should be ignored for being too old - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 1) // should work because it's a new epoch, which will increase the number of available map // outputs, and also finish the stage taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(1), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) @@ -1140,12 +1114,9 @@ class DAGSchedulerSuite // then one executor dies, and a task fails in stage 1 runEvent(ExecutorLost("exec-hostA")) - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), - null, - null, - createFakeTaskInfo(), null)) // so we resubmit stage 0, which completes happily @@ -1155,13 +1126,10 @@ class DAGSchedulerSuite assert(stage0Resubmit.stageAttemptId === 1) val task = stage0Resubmit.tasks(0) assert(task.partitionId === 2) - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( task, Success, - makeMapStatus("hostC", shuffleMapRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostC", shuffleMapRdd.partitions.length))) // now here is where things get tricky : we will now have a task set representing // the second attempt for stage 1, but we *also* have some tasks for the first attempt for @@ -1174,28 +1142,19 @@ class DAGSchedulerSuite // we'll have some tasks finish from the first attempt, and some finish from the second attempt, // so that we actually have all stage outputs, though no attempt has completed all its // tasks - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(3).tasks(0), Success, - makeMapStatus("hostC", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) - runEvent(CompletionEvent( + makeMapStatus("hostC", reduceRdd.partitions.length))) + runEvent(makeCompletionEvent( taskSets(3).tasks(1), Success, - makeMapStatus("hostC", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostC", reduceRdd.partitions.length))) // late task finish from the first attempt - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(2), Success, - makeMapStatus("hostB", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostB", reduceRdd.partitions.length))) // What should happen now is that we submit stage 2. However, we might not see an error // b/c of DAGScheduler's error handling (it tends to swallow errors and just log them). But @@ -1242,21 +1201,21 @@ class DAGSchedulerSuite submit(reduceRdd, Array(0)) // complete some of the tasks from the first stage, on one host - runEvent(CompletionEvent( - taskSets(0).tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.length), null, createFakeTaskInfo(), null)) - runEvent(CompletionEvent( - taskSets(0).tasks(1), Success, - makeMapStatus("hostA", reduceRdd.partitions.length), null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent( + taskSets(0).tasks(0), + Success, + makeMapStatus("hostA", reduceRdd.partitions.length))) + runEvent(makeCompletionEvent( + taskSets(0).tasks(1), + Success, + makeMapStatus("hostA", reduceRdd.partitions.length))) // now that host goes down runEvent(ExecutorLost("exec-hostA")) // so we resubmit those tasks - runEvent(CompletionEvent( - taskSets(0).tasks(0), Resubmitted, null, null, createFakeTaskInfo(), null)) - runEvent(CompletionEvent( - taskSets(0).tasks(1), Resubmitted, null, null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(taskSets(0).tasks(0), Resubmitted, null)) + runEvent(makeCompletionEvent(taskSets(0).tasks(1), Resubmitted, null)) // now complete everything on a different host complete(taskSets(0), Seq( @@ -1884,8 +1843,7 @@ class DAGSchedulerSuite submitMapStage(shuffleDep) val oldTaskSet = taskSets(0) - runEvent(CompletionEvent(oldTaskSet.tasks(0), Success, makeMapStatus("hostA", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(oldTaskSet.tasks(0), Success, makeMapStatus("hostA", 2))) assert(results.size === 0) // Map stage job should not be complete yet // Pretend host A was lost @@ -1895,23 +1853,19 @@ class DAGSchedulerSuite assert(newEpoch > oldEpoch) // Suppose we also get a completed event from task 1 on the same host; this should be ignored - runEvent(CompletionEvent(oldTaskSet.tasks(1), Success, makeMapStatus("hostA", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(oldTaskSet.tasks(1), Success, makeMapStatus("hostA", 2))) assert(results.size === 0) // Map stage job should not be complete yet // A completion from another task should work because it's a non-failed host - runEvent(CompletionEvent(oldTaskSet.tasks(2), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(oldTaskSet.tasks(2), Success, makeMapStatus("hostB", 2))) assert(results.size === 0) // Map stage job should not be complete yet // Now complete tasks in the second task set val newTaskSet = taskSets(1) assert(newTaskSet.tasks.size === 2) // Both tasks 0 and 1 were on on hostA - runEvent(CompletionEvent(newTaskSet.tasks(0), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(newTaskSet.tasks(0), Success, makeMapStatus("hostB", 2))) assert(results.size === 0) // Map stage job should not be complete yet - runEvent(CompletionEvent(newTaskSet.tasks(1), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(newTaskSet.tasks(1), Success, makeMapStatus("hostB", 2))) assert(results.size === 1) // Map stage job should now finally be complete assertDataStructuresEmpty() @@ -1962,5 +1916,15 @@ class DAGSchedulerSuite info } + private def makeCompletionEvent( + task: Task[_], + reason: TaskEndReason, + result: Any, + extraAccumUpdates: Map[Long, Any] = Map[Long, Any](), + taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { + val accumUpdates = task.internalAccumulators.map { a => (a.id, 0L) }.toMap ++ extraAccumUpdates + CompletionEvent(task, reason, result, accumUpdates, taskInfo) + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index f7e16af9d3a9..f08a4ed478d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -17,12 +17,12 @@ package org.apache.spark.scheduler -import org.apache.spark.TaskContext +import org.apache.spark.{InternalAccumulator, TaskContext} class FakeTask( stageId: Int, prefLocs: Seq[TaskLocation] = Nil) - extends Task[Int](stageId, 0, 0, Seq.empty) { + extends Task[Int](stageId, 0, 0, InternalAccumulator.create()) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ecc18fc6e15b..bc93ff8086ee 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,7 +24,6 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.ManualClock class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) @@ -39,8 +38,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { + taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -167,14 +165,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val accumUpdates = taskSet.tasks.head.internalAccumulators.map { a => (a.id, 0L) }.toMap // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) // Tell it the task has finished - manager.handleSuccessfulTask(0, createTaskResult(0)) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdates), 1L) assert(sched.endedTasks(0) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -184,10 +183,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + val accumUpdatesByTask: Array[Map[Long, Long]] = + taskSet.tasks.map { _.internalAccumulators.map { a => (a.id, 0L) }.toMap } // First three offers should all find tasks for (i <- 0 until 3) { - var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -198,14 +199,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) // Finish the first two tasks - manager.handleSuccessfulTask(0, createTaskResult(0)) - manager.handleSuccessfulTask(1, createTaskResult(1)) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0)), 1L) + manager.handleSuccessfulTask(1, createTaskResult(1, accumUpdatesByTask(1)), 1L) assert(sched.endedTasks(0) === Success) assert(sched.endedTasks(1) === Success) assert(!sched.finishedManagers.contains(manager)) // Finish the last task - manager.handleSuccessfulTask(2, createTaskResult(2)) + manager.handleSuccessfulTask(2, createTaskResult(2, accumUpdatesByTask(2)), 1L) assert(sched.endedTasks(2) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -620,7 +621,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // multiple 1k result val r = sc.makeRDD(0 until 10, 10).map(genBytes(1024)).collect() - assert(10 === r.size ) + assert(10 === r.size) // single 10M result val thrown = intercept[SparkException] {sc.makeRDD(genBytes(10 << 20)(0), 1).collect()} @@ -786,8 +787,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(TaskLocation("executor_host1_3") === ExecutorCacheTaskLocation("host1", "3")) } - def createTaskResult(id: Int): DirectTaskResult[Int] = { + private def createTaskResult( + id: Int, + accumUpdates: Map[Long, Any] = Map[Long, Any]()): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) + new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index e6c13fb2c7c8..50453bb6d567 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -240,7 +240,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val taskFailedReasons = Seq( Resubmitted, new FetchFailed(null, 0, 0, 0, "ignored"), - ExceptionFailure("Exception", "description", null, null, None, None), + ExceptionFailure("Exception", "description", null, null, None), TaskResultLost, TaskKilled, ExecutorLostFailure("0", true, Some("Induced failure")), diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index ad33c6833985..ba31c53dacf5 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -142,7 +142,8 @@ class JsonProtocolSuite extends SparkFunSuite { "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskEndReason - val exceptionFailure = new ExceptionFailure(exception, None) + val exceptionFailure = new ExceptionFailure( + exception, scala.collection.immutable.Map[Long, Any]()) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) @@ -167,8 +168,7 @@ class JsonProtocolSuite extends SparkFunSuite { * ============================== */ test("ExceptionFailure backward compatibility") { - val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, null, - None, None) + val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, null, None) val oldEvent = JsonProtocol.taskEndReasonToJson(exceptionFailure) .removeField({ _._1 == "Full Stack Trace" }) assertEquals(exceptionFailure, JsonProtocol.taskEndReasonFromJson(oldEvent)) @@ -638,8 +638,7 @@ class JsonProtocolSuite extends SparkFunSuite { private def assertJsonStringEquals(json1: String, json2: String) { val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") - assert(formatJsonString(json1) === formatJsonString(json2), - s"input ${formatJsonString(json1)} got ${formatJsonString(json2)}") + assert(formatJsonString(json1) === formatJsonString(json2)) } private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { From 2add53f8f83eeec9dd9338a18c32121ddded8afc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Jan 2016 20:08:14 -0800 Subject: [PATCH 18/65] Restore accumulator serialization semantics In the previous commit, we made accumulator communication one-way again, which is the same as before this patch, so we restored all the semantics involved in serializing accumulators as before. Note: tests are still failing because of a duplicate accumulator name in some SQL things. Run `DataFrameCallbackSuite` for more detail. --- .../scala/org/apache/spark/Accumulators.scala | 26 +++++----- .../apache/spark/executor/TaskMetrics.scala | 48 +++++++++---------- .../org/apache/spark/AccumulatorSuite.scala | 33 ++++++------- 3 files changed, 54 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index c83738281bb4..636d6a60efde 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -68,8 +68,9 @@ class Accumulable[R, T] private[spark] ( val id: Long = Accumulators.newId() - @volatile private var value_ : R = initialValue - val zero = param.zero(initialValue) + @volatile @transient private var value_ : R = initialValue // Current value on driver + val zero = param.zero(initialValue) // Zero value to be passed to executors + private var deserialized = false // Avoid leaking accumulators on executors if (isDriver) { @@ -115,10 +116,11 @@ class Accumulable[R, T] private[spark] ( * Access the accumulator's current value; only allowed on driver. */ def value: R = { - if (!isDriver) { + if (!deserialized) { + value_ + } else { throw new UnsupportedOperationException("Can't read accumulator value in task") } - value_ } /** @@ -135,23 +137,19 @@ class Accumulable[R, T] private[spark] ( /** * Set the accumulator's value; only allowed on driver. */ - def value_= (newValue: R): Unit = { - if (!isDriver) { + def value_= (newValue: R) { + if (!deserialized) { + value_ = newValue + } else { throw new UnsupportedOperationException("Can't assign accumulator value in task") } - value_ = newValue } /** - * Set the accumulator's value. + * Set the accumulator's value. For internal use only. */ private[spark] def setValue(newValue: R): Unit = { value_ = newValue } - /** - * Reset the accumulator's value to zero. - */ - private[spark] def resetValue(): Unit = { setValue(zero) } - /** * Whether we are on the driver or the executors. * Note: in local mode, this will inevitably return true even if we're on the executor. @@ -163,6 +161,8 @@ class Accumulable[R, T] private[spark] ( // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() + value_ = zero + deserialized = true // Automatically register the accumulator when it is deserialized with the task closure. val taskContext = TaskContext.get() if (taskContext != null) { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 7e3559e5c1f7..39e9da8d6139 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -80,37 +80,37 @@ class TaskMetrics private[spark] ( /** * Time taken on the executor to deserialize this task. */ - def executorDeserializeTime: Long = _executorDeserializeTime.value + def executorDeserializeTime: Long = _executorDeserializeTime.localValue /** * Time the executor spends actually running the task (including fetching shuffle data). */ - def executorRunTime: Long = _executorRunTime.value + def executorRunTime: Long = _executorRunTime.localValue /** * The number of bytes this task transmitted back to the driver as the TaskResult. */ - def resultSize: Long = _resultSize.value + def resultSize: Long = _resultSize.localValue /** * Amount of time the JVM spent in garbage collection while executing this task. */ - def jvmGCTime: Long = _jvmGCTime.value + def jvmGCTime: Long = _jvmGCTime.localValue /** * Amount of time spent serializing the task result. */ - def resultSerializationTime: Long = _resultSerializationTime.value + def resultSerializationTime: Long = _resultSerializationTime.localValue /** * The number of in-memory bytes spilled by this task. */ - def memoryBytesSpilled: Long = _memoryBytesSpilled.value + def memoryBytesSpilled: Long = _memoryBytesSpilled.localValue /** * The number of on-disk bytes spilled by this task. */ - def diskBytesSpilled: Long = _diskBytesSpilled.value + def diskBytesSpilled: Long = _diskBytesSpilled.localValue /** * Peak memory used by internal data structures created during shuffles, aggregations and @@ -118,7 +118,7 @@ class TaskMetrics private[spark] ( * across all such data structures created in this task. For SQL jobs, this only tracks all * unsafe operators and ExternalSort. */ - def peakExecutionMemory: Long = _peakExecutionMemory.value + def peakExecutionMemory: Long = _peakExecutionMemory.localValue private[spark] def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) private[spark] def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) @@ -137,8 +137,6 @@ class TaskMetrics private[spark] ( val name = a.name.get assert(a.isInternal, s"internal accumulator $name (${a.id}) is not marked as 'internal'") assert(!accumMap.contains(name), s"found duplicate internal accumulator name: $name") - // Note: reset the value here so we don't double count the values in local mode - a.resetValue() accumMap(name) = a } @@ -292,7 +290,9 @@ class TaskMetrics private[spark] ( /** * Return a map from accumulator ID to the accumulator's latest value in this task. */ - def accumulatorUpdates(): Map[Long, Any] = accumMap.values.map { a => (a.id, a.value) }.toMap + def accumulatorUpdates(): Map[Long, Any] = { + accumMap.values.map { a => (a.id, a.localValue) }.toMap + } /** * Storage statuses of any blocks that have been updated as a result of this task. @@ -363,12 +363,12 @@ class InputMetrics private ( /** * Total number of bytes read. */ - def bytesRead: Long = _bytesRead.value + def bytesRead: Long = _bytesRead.localValue /** * Total number of records read. */ - def recordsRead: Long = _recordsRead.value + def recordsRead: Long = _recordsRead.localValue private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) @@ -398,12 +398,12 @@ class OutputMetrics private ( /** * Total number of bytes written. */ - def bytesWritten: Long = _bytesWritten.value + def bytesWritten: Long = _bytesWritten.localValue /** * Total number of records written. */ - def recordsWritten: Long = _recordsWritten.value + def recordsWritten: Long = _recordsWritten.localValue private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) @@ -450,34 +450,34 @@ class ShuffleReadMetrics private ( /** * Number of remote blocks fetched in this shuffle by this task. */ - def remoteBlocksFetched: Long = _remoteBlocksFetched.value + def remoteBlocksFetched: Long = _remoteBlocksFetched.localValue /** * Number of local blocks fetched in this shuffle by this task. */ - def localBlocksFetched: Long = _localBlocksFetched.value + def localBlocksFetched: Long = _localBlocksFetched.localValue /** * Total number of remote bytes read from the shuffle by this task. */ - def remoteBytesRead: Long = _remoteBytesRead.value + def remoteBytesRead: Long = _remoteBytesRead.localValue /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ - def localBytesRead: Long = _localBytesRead.value + def localBytesRead: Long = _localBytesRead.localValue /** * Time the task spent waiting for remote shuffle blocks. This only includes the time * blocking on shuffle input data. For instance if block B is being fetched while the task is * still not finished processing block A, it is not considered to be blocking on block B. */ - def fetchWaitTime: Long = _fetchWaitTime.value + def fetchWaitTime: Long = _fetchWaitTime.localValue /** * Total number of records read from the shuffle by this task. */ - def recordsRead: Long = _recordsRead.value + def recordsRead: Long = _recordsRead.localValue /** * Total bytes fetched in the shuffle by this task (both remote and local). @@ -539,17 +539,17 @@ class ShuffleWriteMetrics private ( /** * Number of bytes written for the shuffle by this task. */ - def bytesWritten: Long = _bytesWritten.value + def bytesWritten: Long = _bytesWritten.localValue /** * Total number of records written to the shuffle by this task. */ - def recordsWritten: Long = _recordsWritten.value + def recordsWritten: Long = _recordsWritten.localValue /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. */ - def shuffleWriteTime: Long = _shuffleWriteTime.value + def shuffleWriteTime: Long = _shuffleWriteTime.localValue private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index f599c9b27d2d..30c901cb1275 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -45,7 +45,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test("basic accumulation") { + test ("basic accumulation"){ sc = new SparkContext("local", "test") val acc : Accumulator[Int] = sc.accumulator(0) @@ -59,16 +59,15 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex longAcc.value should be (210L + maxInt * 20) } - test("value not assignable from tasks") { - // Note: this does not work in local mode because we can't tell whether we're on the driver - sc = new SparkContext("local-cluster[1,1,1024]", "test") + test ("value not assignable from tasks") { + sc = new SparkContext("local", "test") val acc : Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) an [Exception] should be thrownBy {d.foreach{x => acc.value = x}} } - test("add value to collection accumulators") { + test ("add value to collection accumulators") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -85,20 +84,22 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test("value not readable in tasks") { + test ("value not readable in tasks") { val maxI = 1000 - // Note: this does not work in local mode because we can't tell whether we're on the driver - sc = new SparkContext("local-cluster[1,1,1024]", "test") - val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val d = sc.parallelize(1 to maxI) - an [SparkException] should be thrownBy { - d.foreach { - x => acc.value += x + for (nThreads <- List(1, 10)) { // test single & multi-threaded + sc = new SparkContext("local[" + nThreads + "]", "test") + val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val d = sc.parallelize(1 to maxI) + an [SparkException] should be thrownBy { + d.foreach { + x => acc.value += x + } } + resetSparkContext() } } - test("collection accumulators") { + test ("collection accumulators") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded @@ -124,7 +125,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test("localValue readable in tasks") { + test ("localValue readable in tasks") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -139,7 +140,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test("garbage collection") { + test ("garbage collection") { // Create an accumulator and let it go out of scope to test that it's properly garbage collected sc = new SparkContext("local", "test") var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) From afe957c5f28a1c24cd49194e60fb309e301aa427 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 7 Jan 2016 13:06:12 -0800 Subject: [PATCH 19/65] Fix semantics of accumulators when tasks fail Currently we still get values for tasks that fail. We should keep this semantics in the new accumulator updates as well. --- .../scala/org/apache/spark/Accumulators.scala | 17 +++++-- .../scala/org/apache/spark/TaskContext.scala | 5 -- .../org/apache/spark/TaskContextImpl.scala | 4 -- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 51 ++++++++++++------- .../org/apache/spark/scheduler/Task.scala | 10 ++-- .../org/apache/spark/AccumulatorSuite.scala | 2 +- 7 files changed, 55 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 636d6a60efde..9d89f546c0fe 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -45,14 +45,19 @@ import org.apache.spark.util.Utils * @param internal if this [[Accumulable]] is internal. Internal [[Accumulable]]s will be reported * to the driver via heartbeats. For internal [[Accumulable]]s, `R` must be * thread safe so that they can be reported correctly. + * @param countFailedValues whether to accumulate values from failed tasks. This is set to true + * for system and time metrics like serialization time or bytes spilled, + * and false for things with absolute values like number of input rows. + * This should be used for internal metrics only. * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ class Accumulable[R, T] private[spark] ( - initialValue: R, + @transient initialValue: R, param: AccumulableParam[R, T], val name: Option[String], - internal: Boolean) + internal: Boolean, + val countFailedValues: Boolean = false) extends Serializable { private[spark] def this( @@ -260,14 +265,18 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `T` + * @param name human-readable name associated with this accumulator + * @param internal whether this accumulator is used internally within Spark only + * @param countFailedValues whether to accumulate values from failed tasks * @tparam T result type */ class Accumulator[T] private[spark] ( @transient private[spark] val initialValue: T, param: AccumulatorParam[T], name: Option[String], - internal: Boolean) - extends Accumulable[T, T](initialValue, param, name, internal) { + internal: Boolean, + override val countFailedValues: Boolean = false) + extends Accumulable[T, T](initialValue, param, name, internal, countFailedValues) { def this(initialValue: T, param: AccumulatorParam[T], name: Option[String]) = { this(initialValue, param, name, false) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 0e366a7bbad2..5eeae251dcc6 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -164,9 +164,4 @@ abstract class TaskContext extends Serializable { */ private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit - /** - * Return the local values of accumulators that belong to this task. The key of the Map is the - * accumulator id and the value of the Map is the latest accumulator local value. - */ - private[spark] def collectAccumulatorValues(): Map[Long, Any] } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 641102d01171..c5da0f86d611 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -115,10 +115,6 @@ private[spark] class TaskContextImpl( _accumulators(a.id) = a } - private[spark] override def collectAccumulatorValues(): Map[Long, Any] = synchronized { - _accumulators.mapValues(_.localValue).toMap - } - // For testing only. private[spark] def findTestAccum(): Option[Accumulator[Long]] = { accumulators diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4546b5426209..e2c9a4568161 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -303,7 +303,7 @@ private[spark] class Executor( m.setExecutorRunTime(System.currentTimeMillis() - taskStart) m.setJvmGCTime(computeTotalGcTime() - startGCTime) } - task.collectAccumulatorUpdates() + task.collectAccumulatorUpdates(taskFailed = true) } else { Map[Long, Any]() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4391587e0e54..f5d067166be9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1031,7 +1031,6 @@ class DAGScheduler( } case stage: ResultStage => - val job = stage.activeJob.get partitionsToCompute.map { id => val p: Int = stage.partitions(id) val part = stage.rdd.partitions(p) @@ -1111,24 +1110,35 @@ class DAGScheduler( /** * Reconstruct [[TaskMetrics]] from accumulator updates. * - * This is needed for posting task end events to listeners. If the task has failed, - * `accumUpdates` may be null, in which case we just return null. + * This is needed for posting task end events to listeners. + * If the task has failed, we may just return null. */ private def reconstructTaskMetrics(task: Task[_], accumUpdates: Map[Long, Any]): TaskMetrics = { - if (accumUpdates != null) { - task.internalAccumulators.foreach { a => - assert(a.name.isDefined, s"internal accumulator ${a.id} is expected to have a name.") - assert(accumUpdates.contains(a.id), - s"missing entry in task accumulator updates: ${a.name.get} (${a.id})") - val stringValue = accumUpdates(a.id).toString - assert(stringValue.forall(_.isDigit), - s"existing value for accumulator ${a.name.get} (${a.id}) was not a number: $stringValue ") - a.setValue(stringValue.toLong) + if (accumUpdates == null) { + return null + } + val (matchingAccums, missingAccums) = task.internalAccumulators.partition { a => + assert(a.name.isDefined, s"internal accumulator ${a.id} is expected to have a name.") + accumUpdates.contains(a.id) + } + // We create the internal accumulators on the driver and expect the executor to send back + // accumulator values with matching IDs. If the task has failed, then there may be missing + // accumulator values, in which case we just return null. + if (missingAccums.nonEmpty) { + val missingValuesString = missingAccums.map(_.name.get).mkString("[", ",", "]") + logWarning(s"Not reconstructing metrics for task ${task.partitionId} because its " + + s"accumulator updates had missing values $missingValuesString. This could happen " + + s"if the task had failed.") + return null + } + matchingAccums.foreach { a => + accumUpdates(a.id) match { + case l: java.lang.Long => a.setValue(l) + case x => throw new SparkException(s"existing value for accumulator " + + s"${a.name.get} (${a.id}) was of unexpected type: $x (${x.getClass.getName}})") } - new TaskMetrics(task.internalAccumulators) - } else { - null } + new TaskMetrics(task.internalAccumulators) } /** @@ -1140,8 +1150,9 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - // Executors only send accumulator updates back to the driver, not TaskMetrics. However, - // we still need TaskMetrics to post task end events to listeners, so reconstruct them here. + // Executors only send accumulator updates back to the driver, not TaskMetrics. However, we + // still need a TaskMetrics to post "task end" events to listeners, so reconstruct them here. + // Note: this may be null if the task failed. val taskMetrics = reconstructTaskMetrics(task, event.accumUpdates) outputCommitCoordinator.taskCompleted( @@ -1316,7 +1327,11 @@ class DAGScheduler( // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits case exceptionFailure: ExceptionFailure => - // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + // Note: on task failure, the executor only sends back only accumulators whose values + // still matter even when a task fails, e.g. number of bytes spilled to disk. This only + // applies to internal metrics for now. + // TODO: add a test for this + updateAccumulators(event) case TaskResultLost => // Do nothing here; the TaskScheduler handles these failures and resubmits the task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index be02f179bef0..106f5049fdd7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -130,11 +130,15 @@ private[spark] abstract class Task[T]( protected var _executorDeserializeTime: Long = 0 /** - * Collect the latest values of accumulators used in this task. + * Collect the latest values of accumulators used in this task. If the task failed, + * filter out the accumulators whose values should not be included on failures. */ - def collectAccumulatorUpdates(): Map[Long, Any] = { + def collectAccumulatorUpdates(taskFailed: Boolean = false): Map[Long, Any] = { if (context != null) { - context.collectAccumulatorValues() + context.accumulators + .filter { a => !taskFailed || a.countFailedValues } + .map { a => (a.id, a.localValue) } + .toMap } else { Map[Long, Any]() } diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 30c901cb1275..a39dddb682bc 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -164,7 +164,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val accums = InternalAccumulator.create(sc) val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null, accums) val accumulators = taskContext.accumulators - val accumulatorValues = taskContext.collectAccumulatorValues() + val accumulatorValues = taskContext.accumulators.map { a => (a.id, a.value) }.toMap assert(accumulators.size > 0) assert(accumulators.forall(_.isInternal)) val testAccum = taskContext.findTestAccum() From c7240f3dd4490e81905131841596eee72be72898 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 7 Jan 2016 13:38:55 -0800 Subject: [PATCH 20/65] Fix a few more tests There are a few places where we passed in empty internal accumulators to TaskContextImpl, so the TaskMetrics creation would fail. These are now fixed. --- core/src/main/scala/org/apache/spark/TaskContext.scala | 2 +- core/src/test/scala/org/apache/spark/CacheManagerSuite.scala | 5 +++-- .../scala/org/apache/spark/memory/MemoryTestingUtils.scala | 4 ++-- .../sql/execution/UnsafeFixedWidthAggregationMapSuite.scala | 5 +++-- .../spark/sql/execution/UnsafeKVExternalSorterSuite.scala | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 5eeae251dcc6..6fc36fdcb8d6 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -64,7 +64,7 @@ object TaskContext { * An empty task context that does not represent an actual task. */ private[spark] def empty(): TaskContextImpl = { - new TaskContextImpl(0, 0, 0, 0, null, null, Seq.empty) + new TaskContextImpl(0, 0, 0, 0, null, null, InternalAccumulator.create()) } } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index cb8bd04e496a..c1035af6ebd9 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -21,7 +21,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -86,7 +86,8 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before // Local computation should not persist the resulting value, so don't expect a put(). when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - val context = new TaskContextImpl(0, 0, 0, 0, null, null, Seq.empty, runningLocally = true) + val context = new TaskContextImpl( + 0, 0, 0, 0, null, null, InternalAccumulator.create(sc), runningLocally = true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } 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 0e60cc8e7787..b81c95a37380 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.memory -import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} +import org.apache.spark.{InternalAccumulator, SparkEnv, TaskContext, TaskContextImpl} /** * Helper methods for mocking out memory-management-related classes in tests. @@ -32,6 +32,6 @@ object MemoryTestingUtils { attemptNumber = 0, taskMemoryManager = taskMemoryManager, metricsSystem = env.metricsSystem, - internalAccumulators = Seq.empty) + internalAccumulators = InternalAccumulator.create()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 9c258cb31f46..a37147282fb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -23,13 +23,14 @@ import scala.util.control.NonFatal import org.scalatest.Matchers -import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types.StructField /** * Test suite for [[UnsafeFixedWidthAggregationMap]]. @@ -72,7 +73,7 @@ class UnsafeFixedWidthAggregationMapSuite attemptNumber = 0, taskMemoryManager = taskMemoryManager, metricsSystem = null, - internalAccumulators = Seq.empty)) + internalAccumulators = InternalAccumulator.create())) try { f diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index 95c9550aebb0..ff38cb02040d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -118,7 +118,7 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { attemptNumber = 0, taskMemoryManager = taskMemMgr, metricsSystem = null, - internalAccumulators = Seq.empty)) + internalAccumulators = InternalAccumulator.create())) val sorter = new UnsafeKVExternalSorter( keySchema, valueSchema, SparkEnv.get.blockManager, pageSize) From fa086c36647520f6e04622003a8e795bbffd98ad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 7 Jan 2016 17:24:35 -0800 Subject: [PATCH 21/65] Fix SQL UI Before this commit the SQL UI would not display any accumulators. This is because it is powered by the SQLListener, which reads accumulators from TaskMetrics. However, we did not update the accumulator values before posting the TaskMetrics, so the UI never saw the updates from the tasks. This commit also fixes a few related test failures. --- .../scala/org/apache/spark/Accumulators.scala | 25 +++-- .../apache/spark/executor/TaskMetrics.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 91 +++++++++++-------- .../org/apache/spark/scheduler/Task.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- .../spark/sql/execution/SparkPlan.scala | 1 + .../spark/sql/execution/SparkPlanInfo.scala | 3 +- .../sql/execution/metric/SQLMetrics.scala | 21 ++++- 10 files changed, 100 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 9d89f546c0fe..13dd907daa21 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -39,6 +39,9 @@ import org.apache.spark.util.Utils * * Operations are not thread-safe. * + * Note: all internal accumulators used within a task must have unique names because we access + * them by name in [[org.apache.spark.executor.TaskMetrics]]. + * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` * @param name human-readable name for use in Spark's web UI @@ -362,19 +365,27 @@ private[spark] object Accumulators extends Logging { def add(values: Map[Long, Any]): Unit = synchronized { for ((id, value) <- values) { if (originals.contains(id)) { - // Since we are now storing weak references, we must check whether the underlying data - // is valid. - originals(id).get match { - case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] ++= value - case None => - throw new IllegalAccessError("Attempted to access garbage collected Accumulator.") - } + getAccum(id).foreach { _.asInstanceOf[Accumulable[Any, Any]] ++= value } } else { logWarning(s"Ignoring accumulator update for unknown accumulator id $id") } } } + /** + * Return the accumulator registered with the given ID, if any. + */ + def getAccum(id: Long): Option[Accumulable[_, _]] = { + originals.get(id).map { weakRef => + // Since we are storing weak references, we must check whether the underlying data is valid. + weakRef.get match { + case Some(accum) => accum + case None => + throw new IllegalAccessError("Attempted to access garbage collected Accumulator.") + } + } + } + def clear(): Unit = synchronized { originals.clear() } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 39e9da8d6139..3c49cc0d426a 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils */ @DeveloperApi class TaskMetrics private[spark] ( - initialAccums: Seq[Accumulator[Long]], + initialAccums: Seq[Accumulable[_, _]], val hostname: String = TaskMetrics.getCachedHostName) extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f5d067166be9..50c3e8a5299d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1110,35 +1110,36 @@ class DAGScheduler( /** * Reconstruct [[TaskMetrics]] from accumulator updates. * - * This is needed for posting task end events to listeners. - * If the task has failed, we may just return null. + * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. + * However, we need the latter to post task end events to listeners, so we need to + * reconstruct the metrics here on the driver. + * + * Note: If the task failed, we may return null after attempting to reconstruct the + * [[TaskMetrics]] in vain. */ private def reconstructTaskMetrics(task: Task[_], accumUpdates: Map[Long, Any]): TaskMetrics = { if (accumUpdates == null) { return null } - val (matchingAccums, missingAccums) = task.internalAccumulators.partition { a => - assert(a.name.isDefined, s"internal accumulator ${a.id} is expected to have a name.") - accumUpdates.contains(a.id) - } - // We create the internal accumulators on the driver and expect the executor to send back - // accumulator values with matching IDs. If the task has failed, then there may be missing - // accumulator values, in which case we just return null. + // We created the internal accumulators on the driver and expect the executor to send back + // accumulator values with matching IDs. If there are missing values, we cannot reconstruct + // the original TaskMetrics, so just return null. This might happen if the task failed. + val missingAccums = task.initialAccumulators.filter { a => !accumUpdates.contains(a.id) } if (missingAccums.nonEmpty) { val missingValuesString = missingAccums.map(_.name.get).mkString("[", ",", "]") logWarning(s"Not reconstructing metrics for task ${task.partitionId} because its " + - s"accumulator updates had missing values $missingValuesString. This could happen " + - s"if the task had failed.") + s"accumulator updates had missing values: $missingValuesString. This could happen " + + s"if the task failed.") return null } - matchingAccums.foreach { a => - accumUpdates(a.id) match { - case l: java.lang.Long => a.setValue(l) - case x => throw new SparkException(s"existing value for accumulator " + - s"${a.name.get} (${a.id}) was of unexpected type: $x (${x.getClass.getName}})") + // TaskMetrics is not concerned with user accumulators + val internalAccums = accumUpdates.map { case (id, _) => + Accumulators.getAccum(id).getOrElse { + throw new SparkException(s"task ${task.partitionId} returned " + + s"accumulator $id that was not registered on the driver.") } - } - new TaskMetrics(task.internalAccumulators) + }.filter(_.isInternal) + new TaskMetrics(internalAccums.toSeq) } /** @@ -1150,9 +1151,37 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - // Executors only send accumulator updates back to the driver, not TaskMetrics. However, we - // still need a TaskMetrics to post "task end" events to listeners, so reconstruct them here. - // Note: this may be null if the task failed. + // Note: the following events must occur in this order: + // (1) Update accumulator values based on updates from this task + // (2) Reconstruct TaskMetrics + // (3) Post SparkListenerTaskEnd event + // (4) Post SparkListenerStageCompleted / SparkListenerJobEnd event + + // Update accumulator values based on updates from this task. + // Note: we must do this before reconstructing TaskMetrics, otherwise the TaskMetrics + // will not have updated accumulator values. This is needed for the SQL UI, for instance. + if (stageIdToStage.contains(stageId)) { + val stage = stageIdToStage(stageId) + // We should should update registered accumulators if this task succeeded or failed with + // an exception. In the latter case executors may still send back some accumulators, + // so we should try our best to collect the values. + val shouldUpdateAccums = event.reason match { + case Success => + task match { + case rt: ResultTask[_, _] => + // This being true means the job has not finished yet + stage.asInstanceOf[ResultStage].activeJob.isDefined + case smt: ShuffleMapTask => true + } + case _: ExceptionFailure => true + case _ => false + } + if (shouldUpdateAccums) { + updateAccumulators(event) + } + } + + // Reconstruct task metrics. Note: this may be null if the task failed. val taskMetrics = reconstructTaskMetrics(task, event.accumUpdates) outputCommitCoordinator.taskCompleted( @@ -1161,13 +1190,9 @@ class DAGScheduler( event.taskInfo.attemptNumber, // this is a task attempt number event.reason) - // The success case is dealt with separately below, since we need to compute accumulator - // updates before posting. - if (event.reason != Success) { - val attemptId = task.stageAttemptId - listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason, - event.taskInfo, taskMetrics)) - } + // Post task end event + listenerBus.post(SparkListenerTaskEnd( + stageId, task.stageAttemptId, taskType, event.reason, event.taskInfo, taskMetrics)) if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. @@ -1177,8 +1202,6 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) event.reason match { case Success => - listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, - event.reason, event.taskInfo, taskMetrics)) stage.pendingPartitions -= task.partitionId task match { case rt: ResultTask[_, _] => @@ -1188,7 +1211,6 @@ class DAGScheduler( resultStage.activeJob match { case Some(job) => if (!job.finished(rt.outputId)) { - updateAccumulators(event) job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -1215,7 +1237,6 @@ class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] - updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) @@ -1327,11 +1348,7 @@ class DAGScheduler( // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits case exceptionFailure: ExceptionFailure => - // Note: on task failure, the executor only sends back only accumulators whose values - // still matter even when a task fails, e.g. number of bytes spilled to disk. This only - // applies to internal metrics for now. - // TODO: add a test for this - updateAccumulators(event) + // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case TaskResultLost => // Do nothing here; the TaskScheduler handles these failures and resubmits the task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 106f5049fdd7..2ca3850b2d94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -47,7 +47,7 @@ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - val internalAccumulators: Seq[Accumulator[Long]]) extends Serializable { + val initialAccumulators: Seq[Accumulator[Long]]) extends Serializable { /** * The key of the Map is the accumulator id and the value of the Map is the latest accumulator @@ -74,7 +74,7 @@ private[spark] abstract class Task[T]( attemptNumber, taskMemoryManager, metricsSystem, - internalAccumulators, + initialAccumulators, runningLocally = false) TaskContext.setTaskContext(context) taskThread = Thread.currentThread() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e3abc12236d0..bfc9559d1567 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -654,7 +654,7 @@ private[spark] class TaskSetManager( val index = taskInfos(tid).index val task = tasks(index) val accumName = InternalAccumulator.RESULT_SIZE - val resultSizeAccum = task.internalAccumulators.find { a => a.name == Some(accumName) } + val resultSizeAccum = task.initialAccumulators.find { a => a.name == Some(accumName) } assert(resultSizeAccum.isDefined, s"did not find accumulator called '$accumName' in task") val resultSizeAccumId = resultSizeAccum.get.id assert(result.accumUpdates.contains(resultSizeAccumId), 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 74975b7fe51e..f1dd7ce8e696 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1922,7 +1922,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou result: Any, extraAccumUpdates: Map[Long, Any] = Map[Long, Any](), taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { - val accumUpdates = task.internalAccumulators.map { a => (a.id, 0L) }.toMap ++ extraAccumUpdates + val accumUpdates = task.initialAccumulators.map { a => (a.id, 0L) }.toMap ++ extraAccumUpdates CompletionEvent(task, reason, result, accumUpdates, taskInfo) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index bc93ff8086ee..24f275e916bd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -165,7 +165,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - val accumUpdates = taskSet.tasks.head.internalAccumulators.map { a => (a.id, 0L) }.toMap + val accumUpdates = taskSet.tasks.head.initialAccumulators.map { a => (a.id, 0L) }.toMap // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have @@ -184,7 +184,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) val accumUpdatesByTask: Array[Map[Long, Long]] = - taskSet.tasks.map { _.internalAccumulators.map { a => (a.id, 0L) }.toMap } + taskSet.tasks.map { _.initialAccumulators.map { a => (a.id, 0L) }.toMap } // First three offers should all find tasks for (i <- 0 until 3) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 21a6fba9078d..7d2de541b59f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -74,6 +74,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Return all metrics containing metrics of this SparkPlan. + * Key is the internal name used to identify this metric, unique within this [[SparkPlan]]. */ private[sql] def metrics: Map[String, SQLMetric[_, _]] = Map.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 4f750ad13ab8..cfa896835823 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -37,8 +37,7 @@ private[sql] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val metrics = plan.metrics.toSeq.map { case (key, metric) => - new SQLMetricInfo(metric.name.getOrElse(key), metric.id, - Utils.getFormattedClassName(metric.param)) + new SQLMetricInfo(metric.displayName, metric.id, Utils.getFormattedClassName(metric.param)) } val children = plan.children.map(fromSparkPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 52735c9d7f8c..7816f6d53637 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -17,18 +17,30 @@ package org.apache.spark.sql.execution.metric +import java.util.concurrent.atomic.AtomicLong + import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} import org.apache.spark.util.Utils + /** * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. * - * An implementation of SQLMetric should override `+=` and `add` to avoid boxing. + * An implementation of [[SQLMetric]] should override `+=` and `add` to avoid boxing + * + * Note: each [[SQLMetric]] must have a unique name because + * [[org.apache.spark.executor.TaskMetrics]] accesses internal accumulators by name and + * there may be multiple [[SQLMetric]]s per task. Since we never display the names of the + * underlying accumulators on the UI, it is OK to append a GUID to each one. */ private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( name: String, val param: SQLMetricParam[R, T]) - extends Accumulable[R, T](param.zero, param, Some(name), true) { + extends Accumulable[R, T]( + param.zero, param, Some(name + SQLMetrics.nextId.getAndIncrement.toString), internal = true) { + + /** Name of metric to display on the UI. */ + def displayName: String = name def reset(): Unit = { this.value = param.zero @@ -126,6 +138,11 @@ private object StaticsLongSQLMetricParam extends LongSQLMetricParam( private[sql] object SQLMetrics { + /** + * A unique ID used in each [[SQLMetrics]] for the underlying accumulator's name. + */ + private[metric] val nextId = new AtomicLong(0) + private def createLongMetric( sc: SparkContext, name: String, From 361442e30ae5938a23e93a276feae1112ce816a0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 7 Jan 2016 18:52:28 -0800 Subject: [PATCH 22/65] Clean up: lift odd unique name requirement Now internal accumulators no longer need to have unique names. This was an unnecessary hack for the SQL accumulators that can be reverted through some clean ups. --- .../scala/org/apache/spark/Accumulators.scala | 6 +- .../scala/org/apache/spark/TaskContext.scala | 7 +- .../org/apache/spark/TaskContextImpl.scala | 29 +----- .../apache/spark/executor/TaskMetrics.scala | 98 +++++++++++-------- .../apache/spark/scheduler/DAGScheduler.scala | 3 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 22 ++--- .../org/apache/spark/CacheManagerSuite.scala | 3 +- .../spark/memory/MemoryTestingUtils.scala | 5 +- .../spark/sql/execution/SparkPlan.scala | 1 - .../spark/sql/execution/SparkPlanInfo.scala | 3 +- .../sql/execution/metric/SQLMetrics.scala | 21 +--- .../UnsafeFixedWidthAggregationMapSuite.scala | 3 +- .../UnsafeKVExternalSorterSuite.scala | 3 +- .../execution/UnsafeRowSerializerSuite.scala | 3 +- 15 files changed, 89 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 13dd907daa21..05b47e9a4ba1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -417,9 +417,9 @@ private[spark] object InternalAccumulator { // Names of shuffle write metrics object shuffleWrite { - val BYTES_WRITTEN = "metrics.shuffle.write.shuffleBytesWritten" - val RECORDS_WRITTEN = "metrics.shuffle.write.shuffleRecordsWritten" - val WRITE_TIME = "metrics.shuffle.write.shuffleWriteTime" + val BYTES_WRITTEN = "metrics.shuffle.write.bytesWritten" + val RECORDS_WRITTEN = "metrics.shuffle.write.recordsWritten" + val WRITE_TIME = "metrics.shuffle.write.writeTime" } // Names of output metrics diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 6fc36fdcb8d6..e2d7cd5c1052 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -64,7 +64,7 @@ object TaskContext { * An empty task context that does not represent an actual task. */ private[spark] def empty(): TaskContextImpl = { - new TaskContextImpl(0, 0, 0, 0, null, null, InternalAccumulator.create()) + new TaskContextImpl(0, 0, 0, 0, null, null) } } @@ -153,11 +153,6 @@ abstract class TaskContext extends Serializable { */ private[spark] def taskMemoryManager(): TaskMemoryManager - /** - * All accumulators used in this task. - */ - private[spark] def accumulators: Seq[Accumulable[_, _]] - /** * Register an accumulator that belongs to this task. Accumulators must call this method when * deserializing in executors. diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index c5da0f86d611..9dcae3bf1ec3 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -17,7 +17,6 @@ package org.apache.spark -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics @@ -33,7 +32,7 @@ private[spark] class TaskContextImpl( override val attemptNumber: Int, override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, - internalAccumulators: Seq[Accumulator[Long]], + initialAccumulators: Seq[Accumulator[Long]] = InternalAccumulator.create(), val runningLocally: Boolean = false) extends TaskContext with Logging { @@ -41,16 +40,7 @@ private[spark] class TaskContextImpl( /** * Metrics associated with this task. */ - override val taskMetrics: TaskMetrics = new TaskMetrics(internalAccumulators) - - /** - * All accumulators used in this task indexed by accumulator ID. - */ - @transient private val _accumulators = new mutable.HashMap[Long, Accumulable[_, _]] - - // Register the initial set of internal accumulators. - // Future ones will be registered through `registerAccumulator`. - internalAccumulators.foreach { a => _accumulators(a.id) = a } + override val taskMetrics: TaskMetrics = new TaskMetrics(initialAccumulators) // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] @@ -106,19 +96,8 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - private[spark] def accumulators: Seq[Accumulable[_, _]] = _accumulators.values.toSeq - - private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = synchronized { - if (a.isInternal) { - taskMetrics.registerInternalAccum(a) - } - _accumulators(a.id) = a + private[spark] override def registerAccumulator(a: Accumulable[_, _]): Unit = { + taskMetrics.registerAccumulator(a) } - // For testing only. - private[spark] def findTestAccum(): Option[Accumulator[Long]] = { - accumulators - .find(_.name == Some(InternalAccumulator.TEST_ACCUM)) - .map(_.asInstanceOf[Accumulator[Long]]) - } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 3c49cc0d426a..fa860dd12564 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -19,7 +19,6 @@ package org.apache.spark.executor import java.util.concurrent.ConcurrentHashMap -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} @@ -42,6 +41,13 @@ import org.apache.spark.util.Utils * * So, when adding new fields, take into consideration that the whole object can be serialized for * shipping off at any time to consumers of the SparkListener interface. + * + * TODO: update this comment. + * + * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. + * Each accumulator in this initial set must be named and marked as internal. + * Additional accumulators registered here have no such requirements. + * @param hostname where this task is run. */ @DeveloperApi class TaskMetrics private[spark] ( @@ -61,11 +67,26 @@ class TaskMetrics private[spark] ( } /** - * Mapping from metric name to the corresponding internal accumulator. + * All accumulators registered with this task. + */ + private val accums = new ArrayBuffer[Accumulable[_, _]] + accums ++= initialAccums + + /** + * A map for quickly accessing the initial set of accumulators by name. */ - private val accumMap = new mutable.HashMap[String, Accumulable[_, _]] + private val initialAccumsMap: Map[String, Accumulable[_, _]] = { + initialAccums.map { a => + assert(a.name.isDefined, "initial accumulators passed to TaskMetrics should be named") + val name = a.name.get + assert(a.isInternal, + s"initial accumulator '$name' passed to TaskMetrics should be marked as internal") + (name, a) + }.toMap + } - initialAccums.foreach(registerInternalAccum) + assert(initialAccumsMap.size == initialAccums.size, s"detected duplicate names in initial " + + s"accumulators passed to TaskMetrics:\n ${initialAccums.map(_.name.get).mkString("\n")}") // Each metric is internally represented as an accumulator private val _executorDeserializeTime = getLongAccum(EXECUTOR_DESERIALIZE_TIME) @@ -125,28 +146,42 @@ class TaskMetrics private[spark] ( private[spark] def setResultSize(v: Long) = _resultSize.setValue(v) private[spark] def setJvmGCTime(v: Long) = _jvmGCTime.setValue(v) private[spark] def setResultSerializationTime(v: Long) = _resultSerializationTime.setValue(v) - private[spark] def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) - private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) - private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + private[spark] def incMemoryBytesSpilled(v: Long) = _memoryBytesSpilled.add(v) + private[spark] def incDiskBytesSpilled(v: Long) = _diskBytesSpilled.add(v) + private[spark] def incPeakExecutionMemory(v: Long) = _peakExecutionMemory.add(v) /** - * Register an internal accumulator as a new metric. + * Register an accumulator with this task so we can access its value in [[accumulatorUpdates]]. */ - private[spark] def registerInternalAccum(a: Accumulable[_, _]): Unit = { - assert(a.name.isDefined, s"internal accumulator (${a.id}) is expected to have a name") - val name = a.name.get - assert(a.isInternal, s"internal accumulator $name (${a.id}) is not marked as 'internal'") - assert(!accumMap.contains(name), s"found duplicate internal accumulator name: $name") - accumMap(name) = a + private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit = { + accums += a } /** - * Return a Long accumulator associated with the specified metric, assuming it exists. + * Return all the accumulators used on this task. Note: This is not a copy. */ - private def getLongAccum(name: String): Accumulator[Long] = { - TaskMetrics.getLongAccum(accumMap, name) + private[spark] def accumulators: Seq[Accumulable[_, _]] = accums + + /** + * Get a Long accumulator from the given map by name, assuming it exists. + * Note: this only searches the initial set passed into the constructor. + */ + private[spark] def getLongAccum(name: String): Accumulator[Long] = { + TaskMetrics.getLongAccum(initialAccumsMap, name) } + /** + * Return a map from accumulator ID to the accumulator's latest value in this task. + */ + def accumulatorUpdates(): Map[Long, Any] = accums.map { a => (a.id, a.localValue) }.toMap + + + /** + * Storage statuses of any blocks that have been updated as a result of this task. + */ + // TODO: make me an accumulator; right now this doesn't get sent to the driver. + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + /* ============================ * | OUTPUT METRICS | @@ -165,7 +200,7 @@ class TaskMetrics private[spark] ( */ def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = synchronized { _outputMetrics.getOrElse { - val metrics = new OutputMetrics(writeMethod, accumMap.toMap) + val metrics = new OutputMetrics(writeMethod, initialAccumsMap.toMap) _outputMetrics = Some(metrics) metrics } @@ -190,7 +225,7 @@ class TaskMetrics private[spark] ( private[spark] def registerInputMetrics(readMethod: DataReadMethod): InputMetrics = { synchronized { val metrics = _inputMetrics.getOrElse { - val metrics = new InputMetrics(readMethod, accumMap.toMap) + val metrics = new InputMetrics(readMethod, initialAccumsMap.toMap) _inputMetrics = Some(metrics) metrics } @@ -224,7 +259,7 @@ class TaskMetrics private[spark] ( */ def registerShuffleWriteMetrics(): ShuffleWriteMetrics = synchronized { _shuffleWriteMetrics.getOrElse { - val metrics = new ShuffleWriteMetrics(accumMap.toMap) + val metrics = new ShuffleWriteMetrics(initialAccumsMap.toMap) _shuffleWriteMetrics = Some(metrics) metrics } @@ -270,7 +305,7 @@ class TaskMetrics private[spark] ( */ private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { val agg = _shuffleReadMetrics.getOrElse { - val metrics = new ShuffleReadMetrics(accumMap.toMap) + val metrics = new ShuffleReadMetrics(initialAccumsMap.toMap) _shuffleReadMetrics = Some(metrics) metrics } @@ -282,23 +317,6 @@ class TaskMetrics private[spark] ( agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) } - - /* =========================== * - | OTHER THINGS | - * =========================== */ - - /** - * Return a map from accumulator ID to the accumulator's latest value in this task. - */ - def accumulatorUpdates(): Map[Long, Any] = { - accumMap.values.map { a => (a.id, a.localValue) }.toMap - } - - /** - * Storage statuses of any blocks that have been updated as a result of this task. - */ - // TODO: make me an accumulator - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } @@ -579,10 +597,10 @@ private[spark] object TaskMetrics { } /** - * Return a Long accumulator associated with the specified metric, assuming it exists. + * Get a Long accumulator from the given map by name, assuming it exists. */ def getLongAccum( - accumMap: scala.collection.Map[String, Accumulable[_, _]], + accumMap: Map[String, Accumulable[_, _]], name: String): Accumulator[Long] = { assert(accumMap.contains(name), s"metric '$name' is missing") try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 50c3e8a5299d..46365ce5c329 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1139,6 +1139,7 @@ class DAGScheduler( s"accumulator $id that was not registered on the driver.") } }.filter(_.isInternal) + // TODO: the shuffle metrics and stuff are currently not set. This is failing tests. new TaskMetrics(internalAccums.toSeq) } @@ -1164,7 +1165,7 @@ class DAGScheduler( val stage = stageIdToStage(stageId) // We should should update registered accumulators if this task succeeded or failed with // an exception. In the latter case executors may still send back some accumulators, - // so we should try our best to collect the values. + // so we should try our best to collect the values. TODO: write a test. val shouldUpdateAccums = event.reason match { case Success => task match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 2ca3850b2d94..aab63d057df8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -135,7 +135,7 @@ private[spark] abstract class Task[T]( */ def collectAccumulatorUpdates(taskFailed: Boolean = false): Map[Long, Any] = { if (context != null) { - context.accumulators + context.taskMetrics.accumulators .filter { a => !taskFailed || a.countFailedValues } .map { a => (a.id, a.localValue) } .toMap diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index a39dddb682bc..931b0162f67f 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -161,16 +161,14 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex test("internal accumulators in TaskContext") { sc = new SparkContext("local", "test") - val accums = InternalAccumulator.create(sc) - val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null, accums) - val accumulators = taskContext.accumulators - val accumulatorValues = taskContext.accumulators.map { a => (a.id, a.value) }.toMap + val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null) + val accumulators = taskContext.taskMetrics.accumulators + val accumulatorValues = taskContext.taskMetrics.accumulatorUpdates() assert(accumulators.size > 0) assert(accumulators.forall(_.isInternal)) - val testAccum = taskContext.findTestAccum() - assert(testAccum.isDefined) + val testAccum = taskContext.taskMetrics.getLongAccum(TEST_ACCUM) assert(accumulatorValues.size === accumulators.size) - assert(accumulatorValues.contains(testAccum.get.id)) + assert(accumulatorValues.contains(testAccum.id)) } test("internal accumulators in a stage") { @@ -180,7 +178,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum().get += 1 + TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -215,17 +213,17 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 1 + TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 10 + TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().asInstanceOf[TaskContextImpl].findTestAccum.get += 100 + TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -273,7 +271,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => val taskContext = TaskContext.get() - taskContext.asInstanceOf[TaskContextImpl].findTestAccum().get += 1 + taskContext.taskMetrics.getLongAccum(TEST_ACCUM) += 1 // Fail the first attempts of a subset of the tasks if (failCondition(i) && taskContext.attemptNumber() == 0) { throw new Exception("Failing a task intentionally.") diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index c1035af6ebd9..a601966fb177 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -86,8 +86,7 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before // Local computation should not persist the resulting value, so don't expect a put(). when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - val context = new TaskContextImpl( - 0, 0, 0, 0, null, null, InternalAccumulator.create(sc), runningLocally = true) + val context = new TaskContextImpl(0, 0, 0, 0, null, null, runningLocally = true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } 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 b81c95a37380..2b5e4b80e96a 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.memory -import org.apache.spark.{InternalAccumulator, SparkEnv, TaskContext, TaskContextImpl} +import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} /** * Helper methods for mocking out memory-management-related classes in tests. @@ -31,7 +31,6 @@ object MemoryTestingUtils { taskAttemptId = 0, attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = env.metricsSystem, - internalAccumulators = InternalAccumulator.create()) + metricsSystem = env.metricsSystem) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 7d2de541b59f..21a6fba9078d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -74,7 +74,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Return all metrics containing metrics of this SparkPlan. - * Key is the internal name used to identify this metric, unique within this [[SparkPlan]]. */ private[sql] def metrics: Map[String, SQLMetric[_, _]] = Map.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index cfa896835823..4f750ad13ab8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -37,7 +37,8 @@ private[sql] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val metrics = plan.metrics.toSeq.map { case (key, metric) => - new SQLMetricInfo(metric.displayName, metric.id, Utils.getFormattedClassName(metric.param)) + new SQLMetricInfo(metric.name.getOrElse(key), metric.id, + Utils.getFormattedClassName(metric.param)) } val children = plan.children.map(fromSparkPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 7816f6d53637..9ebd8bfd3d07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -17,30 +17,18 @@ package org.apache.spark.sql.execution.metric -import java.util.concurrent.atomic.AtomicLong - import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} import org.apache.spark.util.Utils - /** * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. * - * An implementation of [[SQLMetric]] should override `+=` and `add` to avoid boxing - * - * Note: each [[SQLMetric]] must have a unique name because - * [[org.apache.spark.executor.TaskMetrics]] accesses internal accumulators by name and - * there may be multiple [[SQLMetric]]s per task. Since we never display the names of the - * underlying accumulators on the UI, it is OK to append a GUID to each one. + * An implementation of SQLMetric should override `+=` and `add` to avoid boxing */ private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( name: String, val param: SQLMetricParam[R, T]) - extends Accumulable[R, T]( - param.zero, param, Some(name + SQLMetrics.nextId.getAndIncrement.toString), internal = true) { - - /** Name of metric to display on the UI. */ - def displayName: String = name + extends Accumulable[R, T](param.zero, param, Some(name), internal = true) { def reset(): Unit = { this.value = param.zero @@ -138,11 +126,6 @@ private object StaticsLongSQLMetricParam extends LongSQLMetricParam( private[sql] object SQLMetrics { - /** - * A unique ID used in each [[SQLMetrics]] for the underlying accumulator's name. - */ - private[metric] val nextId = new AtomicLong(0) - private def createLongMetric( sc: SparkContext, name: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index a37147282fb5..e90ed3de8d0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -72,8 +72,7 @@ class UnsafeFixedWidthAggregationMapSuite taskAttemptId = Random.nextInt(10000), attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = null, - internalAccumulators = InternalAccumulator.create())) + metricsSystem = null)) try { f diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index ff38cb02040d..76c89b008220 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -117,8 +117,7 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { taskAttemptId = 98456, attemptNumber = 0, taskMemoryManager = taskMemMgr, - metricsSystem = null, - internalAccumulators = InternalAccumulator.create())) + metricsSystem = null)) val sorter = new UnsafeKVExternalSorter( keySchema, valueSchema, SparkEnv.get.blockManager, pageSize) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 7438e11ef717..2d92329973b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -113,8 +113,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { (i, converter(Row(i))) } val taskMemoryManager = new TaskMemoryManager(sc.env.memoryManager, 0) - val taskContext = new TaskContextImpl( - 0, 0, 0, 0, taskMemoryManager, null, InternalAccumulator.create(sc)) + val taskContext = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, null) val sorter = new ExternalSorter[Int, UnsafeRow, UnsafeRow]( taskContext, From 5aa6aa1bb60d4dead372b307efb1b3b725b4e409 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 10:59:59 -0800 Subject: [PATCH 23/65] Move smaller metrics classes to their own files for readability. --- .../apache/spark/executor/InputMetrics.scala | 83 ++++++ .../apache/spark/executor/OutputMetrics.scala | 67 +++++ .../spark/executor/ShuffleReadMetrics.scala | 116 ++++++++ .../spark/executor/ShuffleWriteMetrics.scala | 79 ++++++ .../apache/spark/executor/TaskMetrics.scala | 263 ------------------ 5 files changed, 345 insertions(+), 263 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/InputMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala new file mode 100644 index 000000000000..b7c4c32258a7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -0,0 +1,83 @@ +/* + * 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.executor + +import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * Method by which input data was read. Network means that the data was read over the network + * from a remote block manager (which may have stored the data on-disk or in-memory). + */ +@DeveloperApi +object DataReadMethod extends Enumeration with Serializable { + type DataReadMethod = Value + val Memory, Disk, Hadoop, Network = Value +} + + +/** + * :: DeveloperApi :: + * Metrics about reading input data. + */ +@DeveloperApi +class InputMetrics private ( + val readMethod: DataReadMethod.Value, + _bytesRead: Accumulator[Long], + _recordsRead: Accumulator[Long]) + extends Serializable { + + private[executor] def this( + readMethod: DataReadMethod.Value, + accumMap: Map[String, Accumulable[_, _]]) { + this( + readMethod, + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.RECORDS_READ)) + } + + /** + * Create a new [[InputMetrics]] that is not associated with any particular task. + * + * This mainly exists because of SPARK-5225, where we are forced to use a dummy [[InputMetrics]] + * because we want to ignore metrics from a second read method. In the future, we should revisit + * whether this is needed. + * + * A better alternative to use is [[TaskMetrics.registerInputMetrics]]. + */ + private[spark] def this(readMethod: DataReadMethod.Value) { + this( + readMethod, + InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) + } + + /** + * Total number of bytes read. + */ + def bytesRead: Long = _bytesRead.localValue + + /** + * Total number of records read. + */ + def recordsRead: Long = _recordsRead.localValue + + private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) +} diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala new file mode 100644 index 000000000000..e6318f7388d1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -0,0 +1,67 @@ +/* + * 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.executor + +import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * Method by which output data was written. + */ +@DeveloperApi +object DataWriteMethod extends Enumeration with Serializable { + type DataWriteMethod = Value + val Hadoop = Value +} + + +/** + * :: DeveloperApi :: + * Metrics about writing output data. + */ +@DeveloperApi +class OutputMetrics private ( + val writeMethod: DataWriteMethod.Value, + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long]) + extends Serializable { + + private[executor] def this( + writeMethod: DataWriteMethod.Value, + accumMap: Map[String, Accumulable[_, _]]) { + this( + writeMethod, + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.BYTES_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.RECORDS_WRITTEN)) + } + + /** + * Total number of bytes written. + */ + def bytesWritten: Long = _bytesWritten.localValue + + /** + * Total number of records written. + */ + def recordsWritten: Long = _recordsWritten.localValue + + private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) + private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) +} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala new file mode 100644 index 000000000000..64f293e982bf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -0,0 +1,116 @@ +/* + * 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.executor + +import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data read in a given task. + */ +@DeveloperApi +class ShuffleReadMetrics private ( + _remoteBlocksFetched: Accumulator[Long], + _localBlocksFetched: Accumulator[Long], + _remoteBytesRead: Accumulator[Long], + _localBytesRead: Accumulator[Long], + _fetchWaitTime: Accumulator[Long], + _recordsRead: Accumulator[Long]) + extends Serializable { + + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + this( + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) + } + + /** + * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + * + * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. + */ + private[spark] def this() { + this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) + } + + /** + * Number of remote blocks fetched in this shuffle by this task. + */ + def remoteBlocksFetched: Long = _remoteBlocksFetched.localValue + + /** + * Number of local blocks fetched in this shuffle by this task. + */ + def localBlocksFetched: Long = _localBlocksFetched.localValue + + /** + * Total number of remote bytes read from the shuffle by this task. + */ + def remoteBytesRead: Long = _remoteBytesRead.localValue + + /** + * Shuffle data that was read from the local disk (as opposed to from a remote executor). + */ + def localBytesRead: Long = _localBytesRead.localValue + + /** + * Time the task spent waiting for remote shuffle blocks. This only includes the time + * blocking on shuffle input data. For instance if block B is being fetched while the task is + * still not finished processing block A, it is not considered to be blocking on block B. + */ + def fetchWaitTime: Long = _fetchWaitTime.localValue + + /** + * Total number of records read from the shuffle by this task. + */ + def recordsRead: Long = _recordsRead.localValue + + /** + * Total bytes fetched in the shuffle by this task (both remote and local). + */ + def totalBytesRead: Long = remoteBytesRead + localBytesRead + + /** + * Number of blocks fetched in this shuffle by this task (remote or local). + */ + def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched + + private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) + private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) + private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) + private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) + private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + + private[spark] def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) + private[spark] def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) + private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) + private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) + private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) + private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) +} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala new file mode 100644 index 000000000000..0c30bc938d61 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -0,0 +1,79 @@ +/* + * 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.executor + +import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data written in a given task. + */ +@DeveloperApi +class ShuffleWriteMetrics private ( + _bytesWritten: Accumulator[Long], + _recordsWritten: Accumulator[Long], + _shuffleWriteTime: Accumulator[Long]) + extends Serializable { + + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + this( + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), + TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) + } + + /** + * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. + * + * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in + * many places only to merge their values together later. In the future, we should revisit + * whether this is needed. + * + * A better alternative to use is [[TaskMetrics.registerShuffleWriteMetrics]]. + */ + private[spark] def this() { + this(InternalAccumulator.createShuffleWriteAccums().map { a => (a.name.get, a) }.toMap) + } + + /** + * Number of bytes written for the shuffle by this task. + */ + def bytesWritten: Long = _bytesWritten.localValue + + /** + * Total number of records written to the shuffle by this task. + */ + def recordsWritten: Long = _recordsWritten.localValue + + /** + * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. + */ + def shuffleWriteTime: Long = _shuffleWriteTime.localValue + + private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) + private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) + private[spark] def incWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) + private[spark] def decBytesWritten(v: Long): Unit = { + _bytesWritten.setValue(bytesWritten - v) + } + private[spark] def decRecordsWritten(v: Long): Unit = { + _recordsWritten.setValue(recordsWritten - v) + } +} diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index fa860dd12564..824dbb398669 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -28,8 +28,6 @@ import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils -// TODO: make everything here private - /** * :: DeveloperApi :: * Metrics tracked during the execution of a task. @@ -320,267 +318,6 @@ class TaskMetrics private[spark] ( } -/** - * :: DeveloperApi :: - * Method by which input data was read. Network means that the data was read over the network - * from a remote block manager (which may have stored the data on-disk or in-memory). - */ -@DeveloperApi -object DataReadMethod extends Enumeration with Serializable { - type DataReadMethod = Value - val Memory, Disk, Hadoop, Network = Value -} - - -/** - * :: DeveloperApi :: - * Method by which output data was written. - */ -@DeveloperApi -object DataWriteMethod extends Enumeration with Serializable { - type DataWriteMethod = Value - val Hadoop = Value -} - - -/** - * :: DeveloperApi :: - * Metrics about reading input data. - */ -@DeveloperApi -class InputMetrics private ( - val readMethod: DataReadMethod.Value, - _bytesRead: Accumulator[Long], - _recordsRead: Accumulator[Long]) - extends Serializable { - - private[executor] def this( - readMethod: DataReadMethod.Value, - accumMap: Map[String, Accumulable[_, _]]) { - this( - readMethod, - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.RECORDS_READ)) - } - - /** - * Create a new [[InputMetrics]] that is not associated with any particular task. - * - * This mainly exists because of SPARK-5225, where we are forced to use a dummy [[InputMetrics]] - * because we want to ignore metrics from a second read method. In the future, we should revisit - * whether this is needed. - * - * A better alternative to use is [[TaskMetrics.registerInputMetrics]]. - */ - private[spark] def this(readMethod: DataReadMethod.Value) { - this( - readMethod, - InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) - } - - /** - * Total number of bytes read. - */ - def bytesRead: Long = _bytesRead.localValue - - /** - * Total number of records read. - */ - def recordsRead: Long = _recordsRead.localValue - - private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) - private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) -} - - -/** - * :: DeveloperApi :: - * Metrics about writing output data. - */ -@DeveloperApi -class OutputMetrics private ( - val writeMethod: DataWriteMethod.Value, - _bytesWritten: Accumulator[Long], - _recordsWritten: Accumulator[Long]) - extends Serializable { - - private[executor] def this( - writeMethod: DataWriteMethod.Value, - accumMap: Map[String, Accumulable[_, _]]) { - this( - writeMethod, - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.BYTES_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.RECORDS_WRITTEN)) - } - - /** - * Total number of bytes written. - */ - def bytesWritten: Long = _bytesWritten.localValue - - /** - * Total number of records written. - */ - def recordsWritten: Long = _recordsWritten.localValue - - private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) - private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) -} - - -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data read in a given task. - */ -@DeveloperApi -class ShuffleReadMetrics private ( - _remoteBlocksFetched: Accumulator[Long], - _localBlocksFetched: Accumulator[Long], - _remoteBytesRead: Accumulator[Long], - _localBytesRead: Accumulator[Long], - _fetchWaitTime: Accumulator[Long], - _recordsRead: Accumulator[Long]) - extends Serializable { - - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { - this( - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) - } - - /** - * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. - * - * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. - */ - private[spark] def this() { - this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) - } - - /** - * Number of remote blocks fetched in this shuffle by this task. - */ - def remoteBlocksFetched: Long = _remoteBlocksFetched.localValue - - /** - * Number of local blocks fetched in this shuffle by this task. - */ - def localBlocksFetched: Long = _localBlocksFetched.localValue - - /** - * Total number of remote bytes read from the shuffle by this task. - */ - def remoteBytesRead: Long = _remoteBytesRead.localValue - - /** - * Shuffle data that was read from the local disk (as opposed to from a remote executor). - */ - def localBytesRead: Long = _localBytesRead.localValue - - /** - * Time the task spent waiting for remote shuffle blocks. This only includes the time - * blocking on shuffle input data. For instance if block B is being fetched while the task is - * still not finished processing block A, it is not considered to be blocking on block B. - */ - def fetchWaitTime: Long = _fetchWaitTime.localValue - - /** - * Total number of records read from the shuffle by this task. - */ - def recordsRead: Long = _recordsRead.localValue - - /** - * Total bytes fetched in the shuffle by this task (both remote and local). - */ - def totalBytesRead: Long = remoteBytesRead + localBytesRead - - /** - * Number of blocks fetched in this shuffle by this task (remote or local). - */ - def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched - - private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) - private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) - private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) - private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) - private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) - private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) - - private[spark] def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) - private[spark] def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) - private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) - private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) - private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) - private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) -} - - -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data written in a given task. - */ -@DeveloperApi -class ShuffleWriteMetrics private ( - _bytesWritten: Accumulator[Long], - _recordsWritten: Accumulator[Long], - _shuffleWriteTime: Accumulator[Long]) - extends Serializable { - - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { - this( - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) - } - - /** - * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. - * - * A better alternative to use is [[TaskMetrics.registerShuffleWriteMetrics]]. - */ - private[spark] def this() { - this(InternalAccumulator.createShuffleWriteAccums().map { a => (a.name.get, a) }.toMap) - } - - /** - * Number of bytes written for the shuffle by this task. - */ - def bytesWritten: Long = _bytesWritten.localValue - - /** - * Total number of records written to the shuffle by this task. - */ - def recordsWritten: Long = _recordsWritten.localValue - - /** - * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. - */ - def shuffleWriteTime: Long = _shuffleWriteTime.localValue - - private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) - private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) - private[spark] def incWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) - private[spark] def decBytesWritten(v: Long): Unit = { - _bytesWritten.setValue(bytesWritten - v) - } - private[spark] def decRecordsWritten(v: Long): Unit = { - _recordsWritten.setValue(recordsWritten - v) - } -} - - private[spark] object TaskMetrics { private val hostNameCache = new ConcurrentHashMap[String, String]() From 7118be5c16e98bcae7b5effa1ef462dc8f780655 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 11:11:07 -0800 Subject: [PATCH 24/65] Fix SQLQuerySuite A few bugs: (1) In Executor.scala, we updated TaskMetrics after collecting the accumulator values. We should do it the other order. (2) The test utility method of verifying whether peak execution memory is set imposed this requirement on every single job run in the test body. This does not apply for SQL's external sort, however, because one of the jobs does a sample and so does not update peak execution memory. (3) We were getting accumulators from executors that were not registered on the driver. Not exactly sure what the cause is but it could very well have to do with GC on the driver since we use weak references there. We shouldn't crash the scheduler if this happens. --- .../org/apache/spark/executor/Executor.scala | 5 ++- .../apache/spark/scheduler/DAGScheduler.scala | 22 +++++++----- .../org/apache/spark/scheduler/Task.scala | 11 ++---- .../org/apache/spark/AccumulatorSuite.scala | 36 ++++--------------- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- 5 files changed, 28 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e2c9a4568161..23555d663c65 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -210,7 +210,7 @@ private[spark] class Executor( // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() var threwException = true - val (value, accumUpdates) = try { + val value = try { val res = task.run( taskAttemptId = taskId, attemptNumber = attemptNumber, @@ -251,6 +251,9 @@ private[spark] class Executor( m.setResultSerializationTime(afterSerialization - beforeSerialization) } + // Note: accumulator updates must be collected after TaskMetrics is updated + // TODO: add a test + val accumUpdates = task.collectAccumulatorUpdates() val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 46365ce5c329..d58c703665be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1132,15 +1132,21 @@ class DAGScheduler( s"if the task failed.") return null } - // TaskMetrics is not concerned with user accumulators - val internalAccums = accumUpdates.map { case (id, _) => - Accumulators.getAccum(id).getOrElse { - throw new SparkException(s"task ${task.partitionId} returned " + - s"accumulator $id that was not registered on the driver.") - } - }.filter(_.isInternal) // TODO: the shuffle metrics and stuff are currently not set. This is failing tests. - new TaskMetrics(internalAccums.toSeq) + val metrics = new TaskMetrics(task.initialAccumulators) + val registeredAccumIds = task.initialAccumulators.map(_.id).toSet + // Register all remaining accumulators separately because these may not be named. + accumUpdates + .flatMap { case (id, _) => + Accumulators.getAccum(id).orElse { + logWarning(s"Task ${task.partitionId} returned accumulator $id that was " + + s"not registered on the driver.") + None + } + } + .filter { a => !registeredAccumIds.contains(a.id) } + .foreach(metrics.registerAccumulator) + metrics } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index aab63d057df8..9b8015e9493f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -49,12 +49,6 @@ private[spark] abstract class Task[T]( val partitionId: Int, val initialAccumulators: Seq[Accumulator[Long]]) extends Serializable { - /** - * The key of the Map is the accumulator id and the value of the Map is the latest accumulator - * local value. - */ - type AccumulatorUpdates = Map[Long, Any] - /** * Called by [[Executor]] to run this task. * @@ -65,8 +59,7 @@ private[spark] abstract class Task[T]( final def run( taskAttemptId: Long, attemptNumber: Int, - metricsSystem: MetricsSystem) - : (T, AccumulatorUpdates) = { + metricsSystem: MetricsSystem): T = { context = new TaskContextImpl( stageId, partitionId, @@ -82,7 +75,7 @@ private[spark] abstract class Task[T]( kill(interruptThread = false) } try { - (runTask(context), collectAccumulatorUpdates()) + runTask(context) } finally { context.markTaskCompleted() try { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 931b0162f67f..f8f825ae9890 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -312,42 +312,20 @@ private[spark] object AccumulatorSuite { import InternalAccumulator._ /** - * Run one or more Spark jobs and verify that the peak execution memory accumulator - * is updated afterwards. + * Run one or more Spark jobs and verify that in at least one job the peak execution memory + * accumulator is updated afterwards. */ def verifyPeakExecutionMemorySet( sc: SparkContext, testName: String)(testBody: => Unit): Unit = { val listener = new SaveInfoListener sc.addSparkListener(listener) - - // Find next job ID - var nextJobId = -1 - listener.registerJobCompletionCallback { jobId => nextJobId = jobId + 1 } - sc.parallelize(1 to 10).count() - require(nextJobId > 0, "bad test: job ID was not updated even after a count()") - - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { jobId => - if (jobId == nextJobId) { - // The first job is a dummy one to verify that the accumulator does not already exist - val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) - assert(!accums.exists(_.name == PEAK_EXECUTION_MEMORY)) - } else { - // In the subsequent jobs, verify that peak execution memory is updated - val accum = listener.getCompletedStageInfos - .flatMap(_.accumulables.values) - .find(_.name == PEAK_EXECUTION_MEMORY) - .getOrElse { - throw new TestFailedException( - s"peak execution memory accumulator not set in '$testName'", 0) - } - assert(accum.value.toLong > 0) - } - } - // Run the jobs - sc.parallelize(1 to 10).count() testBody + val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) + val isSet = accums.exists { a => a.name == PEAK_EXECUTION_MEMORY && a.value.toLong > 0 } + if (!isSet) { + throw new TestFailedException(s"peak execution memory accumulator not set in '$testName'", 0) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bd987ae1bb03..d5e011e4750d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1666,7 +1666,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("external sorting updates peak execution memory") { AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { - sortTest() + sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } } From 176e91d0abfaed3d8741821787a3732b0b9f3688 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 13:22:21 -0800 Subject: [PATCH 25/65] Remove unused hostname from TaskMetrics --- .../apache/spark/executor/TaskMetrics.scala | 26 ++----------------- .../org/apache/spark/util/JsonProtocol.scala | 4 +-- .../apache/spark/util/JsonProtocolSuite.scala | 7 +---- 3 files changed, 4 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 824dbb398669..742b165a7f50 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,15 +17,12 @@ package org.apache.spark.executor -import java.util.concurrent.ConcurrentHashMap - import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} -import org.apache.spark.util.Utils /** @@ -45,23 +42,15 @@ import org.apache.spark.util.Utils * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. * Each accumulator in this initial set must be named and marked as internal. * Additional accumulators registered here have no such requirements. - * @param hostname where this task is run. */ @DeveloperApi -class TaskMetrics private[spark] ( - initialAccums: Seq[Accumulable[_, _]], - val hostname: String = TaskMetrics.getCachedHostName) - extends Serializable { +class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends Serializable { import InternalAccumulator._ - def this(host: String) { - this(InternalAccumulator.create(), host) - } - // Needed for Java tests def this() { - this(TaskMetrics.getCachedHostName) + this(InternalAccumulator.create()) } /** @@ -319,20 +308,9 @@ class TaskMetrics private[spark] ( private[spark] object TaskMetrics { - private val hostNameCache = new ConcurrentHashMap[String, String]() def empty: TaskMetrics = new TaskMetrics - /** - * Get the hostname from cached data, since hostname is the order of number of nodes in cluster, - * so using cached hostname will decrease the object number and alleviate the GC overhead. - */ - def getCachedHostName: String = { - val host = Utils.localHostName() - val canonicalHost = hostNameCache.putIfAbsent(host, host) - if (canonicalHost != null) canonicalHost else host - } - /** * Get a Long accumulator from the given map by name, assuming it exists. */ 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 7d34106c50cd..f69721a9ffdd 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -326,7 +326,6 @@ private[spark] object JsonProtocol { ("Status" -> blockStatusToJson(status)) }) }.getOrElse(JNothing) - ("Host Name" -> taskMetrics.hostname) ~ ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ ("Executor Run Time" -> taskMetrics.executorRunTime) ~ ("Result Size" -> taskMetrics.resultSize) ~ @@ -705,8 +704,7 @@ private[spark] object JsonProtocol { if (json == JNothing) { return TaskMetrics.empty } - val hostname = (json \ "Host Name").extract[String] - val metrics = new TaskMetrics(hostname) + val metrics = new TaskMetrics metrics.setExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long]) metrics.setExecutorRunTime((json \ "Executor Run Time").extract[Long]) metrics.setResultSize((json \ "Result Size").extract[Long]) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index ba31c53dacf5..81993385a032 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -543,7 +543,6 @@ class JsonProtocolSuite extends SparkFunSuite { } private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { - assert(metrics1.hostname === metrics2.hostname) assert(metrics1.executorDeserializeTime === metrics2.executorDeserializeTime) assert(metrics1.resultSize === metrics2.resultSize) assert(metrics1.jvmGCTime === metrics2.jvmGCTime) @@ -761,7 +760,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput: Boolean, hasOutput: Boolean, hasRecords: Boolean = true) = { - val t = new TaskMetrics("localhost") + val t = new TaskMetrics t.setExecutorDeserializeTime(a) t.setExecutorRunTime(b) t.setResultSize(c) @@ -1029,7 +1028,6 @@ class JsonProtocolSuite extends SparkFunSuite { | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1118,7 +1116,6 @@ class JsonProtocolSuite extends SparkFunSuite { | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1204,7 +1201,6 @@ class JsonProtocolSuite extends SparkFunSuite { | ] | }, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, @@ -1692,7 +1688,6 @@ class JsonProtocolSuite extends SparkFunSuite { | "Stage ID": 2, | "Stage Attempt ID": 3, | "Task Metrics": { - | "Host Name": "localhost", | "Executor Deserialize Time": 300, | "Executor Run Time": 400, | "Result Size": 500, From 7939e1c74d9dd62e383c71c72d9b353c3f6d8dfc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 14:18:17 -0800 Subject: [PATCH 26/65] Reinitialize OutputMetrics et al during reconstruction Such that downstream listeners can access their values. This commit also generalizes the internal accumulator type from Long to anything, since we need to store the read and write methods of InputMetrics and OutputMetrics respectively. --- .../scala/org/apache/spark/Accumulators.scala | 156 +++++++++++------- .../org/apache/spark/TaskContextImpl.scala | 2 +- .../apache/spark/executor/InputMetrics.scala | 27 +-- .../apache/spark/executor/OutputMetrics.scala | 21 ++- .../spark/executor/ShuffleReadMetrics.scala | 12 +- .../spark/executor/ShuffleWriteMetrics.scala | 6 +- .../apache/spark/executor/TaskMetrics.scala | 129 +++++++++------ .../apache/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/ShuffleMapTask.scala | 4 +- .../org/apache/spark/scheduler/Stage.scala | 4 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 12 +- 12 files changed, 223 insertions(+), 156 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 05b47e9a4ba1..ad0abd1ac3cb 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -330,7 +330,10 @@ object AccumulatorParam { def zero(initialValue: Float): Float = 0f } - // TODO: Add AccumulatorParams for other types, e.g. lists and strings + private[spark] implicit object StringAccumulatorParam extends AccumulatorParam[String] { + def addInPlace(t1: String, t2: String): String = t1 + t2 + def zero(initialValue: String): String = "" + } } // TODO: The multi-thread support in accumulators is kind of lame; check @@ -394,114 +397,122 @@ private[spark] object Accumulators extends Logging { private[spark] object InternalAccumulator { + // Prefixes used in names of internal task level metrics + val METRICS_PREFIX = "metrics." + val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." + val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." + val OUTPUT_METRICS_PREFIX = METRICS_PREFIX + "output." + val INPUT_METRICS_PREFIX = METRICS_PREFIX + "input." + // Names of internal task level metrics - val EXECUTOR_DESERIALIZE_TIME = "metrics.executorDeserializeTime" - val EXECUTOR_RUN_TIME = "metrics.executorRunTime" - val RESULT_SIZE = "metrics.resultSize" - val JVM_GC_TIME = "metrics.jvmGCTime" - val RESULT_SERIALIZATION_TIME = "metrics.resultSerializationTime" - val MEMORY_BYTES_SPILLED = "metrics.memoryBytesSpilled" - val DISK_BYTES_SPILLED = "metrics.diskBytesSpilled" - val PEAK_EXECUTION_MEMORY = "metrics.peakExecutionMemory" - val TEST_ACCUM = "metrics.testAccumulator" + val EXECUTOR_DESERIALIZE_TIME = METRICS_PREFIX + "executorDeserializeTime" + val EXECUTOR_RUN_TIME = METRICS_PREFIX + "executorRunTime" + val RESULT_SIZE = METRICS_PREFIX + "resultSize" + val JVM_GC_TIME = METRICS_PREFIX + "jvmGCTime" + val RESULT_SERIALIZATION_TIME = METRICS_PREFIX + "resultSerializationTime" + val MEMORY_BYTES_SPILLED = METRICS_PREFIX + "memoryBytesSpilled" + val DISK_BYTES_SPILLED = METRICS_PREFIX + "diskBytesSpilled" + val PEAK_EXECUTION_MEMORY = METRICS_PREFIX + "peakExecutionMemory" + val TEST_ACCUM = METRICS_PREFIX + "testAccumulator" // Names of shuffle read metrics object shuffleRead { - val REMOTE_BLOCKS_FETCHED = "metrics.shuffle.read.remoteBlocksFetched" - val LOCAL_BLOCKS_FETCHED = "metrics.shuffle.read.localBlocksFetched" - val REMOTE_BYTES_READ = "metrics.shuffle.read.remoteBytesRead" - val LOCAL_BYTES_READ = "metrics.shuffle.read.localBytesRead" - val FETCH_WAIT_TIME = "metrics.shuffle.read.fetchWaitTime" - val RECORDS_READ = "metrics.shuffle.read.recordsRead" + val REMOTE_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "localBlocksFetched" + val REMOTE_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesRead" + val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead" + val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime" + val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead" } // Names of shuffle write metrics object shuffleWrite { - val BYTES_WRITTEN = "metrics.shuffle.write.bytesWritten" - val RECORDS_WRITTEN = "metrics.shuffle.write.recordsWritten" - val WRITE_TIME = "metrics.shuffle.write.writeTime" + val BYTES_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "bytesWritten" + val RECORDS_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "recordsWritten" + val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" } // Names of output metrics object output { - val BYTES_WRITTEN = "metrics.output.bytesWritten" - val RECORDS_WRITTEN = "metrics.output.recordsWritten" + val WRITE_METHOD = OUTPUT_METRICS_PREFIX + "writeMethod" + val BYTES_WRITTEN = OUTPUT_METRICS_PREFIX + "bytesWritten" + val RECORDS_WRITTEN = OUTPUT_METRICS_PREFIX + "recordsWritten" } // Names of input metrics object input { - val BYTES_READ = "metrics.input.bytesRead" - val RECORDS_READ = "metrics.input.recordsRead" - } - - /** - * Create a new internal Long accumulator with the specified name. - */ - private def newMetric(name: String): Accumulator[Long] = { - new Accumulator(0L, AccumulatorParam.LongAccumulatorParam, Some(name), internal = true) + val READ_METHOD = INPUT_METRICS_PREFIX + "readMethod" + val BYTES_READ = INPUT_METRICS_PREFIX + "bytesRead" + val RECORDS_READ = INPUT_METRICS_PREFIX + "recordsRead" } /** * Accumulators for tracking internal metrics. * Note: this method does not register accumulators for cleanup. */ - def create(): Seq[Accumulator[Long]] = { - val metricNames = Seq[String]( - EXECUTOR_DESERIALIZE_TIME, - EXECUTOR_RUN_TIME, - RESULT_SIZE, - JVM_GC_TIME, - RESULT_SERIALIZATION_TIME, - MEMORY_BYTES_SPILLED, - DISK_BYTES_SPILLED, - PEAK_EXECUTION_MEMORY) ++ - sys.props.get("spark.testing").map(_ => TEST_ACCUM).toSeq - metricNames.map(newMetric) ++ - createShuffleReadAccums() ++ - createShuffleWriteAccums() ++ - createInputAccums() ++ - createOutputAccums() + def create(): Seq[Accumulator[_]] = { + val maybeTestAccum = sys.props.get("spark.testing").map(_ => newLongMetric(TEST_ACCUM)).toSeq + Seq[Accumulator[_]]( + newLongMetric(EXECUTOR_DESERIALIZE_TIME), + newLongMetric(EXECUTOR_RUN_TIME), + newLongMetric(RESULT_SIZE), + newLongMetric(JVM_GC_TIME), + newLongMetric(RESULT_SERIALIZATION_TIME), + newLongMetric(MEMORY_BYTES_SPILLED), + newLongMetric(DISK_BYTES_SPILLED), + newLongMetric(PEAK_EXECUTION_MEMORY)) ++ + createShuffleReadAccums() ++ + createShuffleWriteAccums() ++ + createInputAccums() ++ + createOutputAccums() ++ + maybeTestAccum } /** * Accumulators for tracking shuffle read metrics. * Note: this method does not register accumulators for cleanup. */ - def createShuffleReadAccums(): Seq[Accumulator[Long]] = { - Seq[String]( - shuffleRead.REMOTE_BLOCKS_FETCHED, - shuffleRead.LOCAL_BLOCKS_FETCHED, - shuffleRead.REMOTE_BYTES_READ, - shuffleRead.LOCAL_BYTES_READ, - shuffleRead.FETCH_WAIT_TIME, - shuffleRead.RECORDS_READ).map(newMetric) + def createShuffleReadAccums(): Seq[Accumulator[_]] = { + Seq[Accumulator[_]]( + newLongMetric(shuffleRead.REMOTE_BLOCKS_FETCHED), + newLongMetric(shuffleRead.LOCAL_BLOCKS_FETCHED), + newLongMetric(shuffleRead.REMOTE_BYTES_READ), + newLongMetric(shuffleRead.LOCAL_BYTES_READ), + newLongMetric(shuffleRead.FETCH_WAIT_TIME), + newLongMetric(shuffleRead.RECORDS_READ)) } /** * Accumulators for tracking shuffle write metrics. * Note: this method does not register accumulators for cleanup. */ - def createShuffleWriteAccums(): Seq[Accumulator[Long]] = { - Seq[String]( - shuffleWrite.BYTES_WRITTEN, - shuffleWrite.RECORDS_WRITTEN, - shuffleWrite.WRITE_TIME).map(newMetric) + def createShuffleWriteAccums(): Seq[Accumulator[_]] = { + Seq[Accumulator[_]]( + newLongMetric(shuffleWrite.BYTES_WRITTEN), + newLongMetric(shuffleWrite.RECORDS_WRITTEN), + newLongMetric(shuffleWrite.WRITE_TIME)) } /** * Accumulators for tracking input metrics. * Note: this method does not register accumulators for cleanup. */ - def createInputAccums(): Seq[Accumulator[Long]] = { - Seq[String](input.BYTES_READ, input.RECORDS_READ).map(newMetric) + def createInputAccums(): Seq[Accumulator[_]] = { + Seq[Accumulator[_]]( + newStringMetric(input.READ_METHOD), + newLongMetric(input.BYTES_READ), + newLongMetric(input.RECORDS_READ)) } /** * Accumulators for tracking output metrics. * Note: this method does not register accumulators for cleanup. */ - private def createOutputAccums(): Seq[Accumulator[Long]] = { - Seq[String](output.BYTES_WRITTEN, output.RECORDS_WRITTEN).map(newMetric) + private def createOutputAccums(): Seq[Accumulator[_]] = { + Seq[Accumulator[_]]( + newStringMetric(output.WRITE_METHOD), + newLongMetric(output.BYTES_WRITTEN), + newLongMetric(output.RECORDS_WRITTEN)) } /** @@ -511,11 +522,28 @@ private[spark] object InternalAccumulator { * add to the same set of accumulators. We do this to report the distribution of accumulator * values across all tasks within each stage. */ - def create(sc: SparkContext): Seq[Accumulator[Long]] = { + def create(sc: SparkContext): Seq[Accumulator[_]] = { val accums = create() accums.foreach { accum => sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) } accums } + + + /** + * Create a new internal Long accumulator with the specified name. + */ + private def newLongMetric(name: String): Accumulator[Long] = { + new Accumulator[Long](0L, AccumulatorParam.LongAccumulatorParam, Some(name), internal = true) + } + + /** + * Create a new internal String accumulator with the specified name. + */ + private def newStringMetric(name: String): Accumulator[String] = { + new Accumulator[String]( + "", AccumulatorParam.StringAccumulatorParam, Some(name), internal = true) + } + } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 9dcae3bf1ec3..fff9aa5d8cad 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -32,7 +32,7 @@ private[spark] class TaskContextImpl( override val attemptNumber: Int, override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, - initialAccumulators: Seq[Accumulator[Long]] = InternalAccumulator.create(), + initialAccumulators: Seq[Accumulator[_]] = InternalAccumulator.create(), val runningLocally: Boolean = false) extends TaskContext with Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index b7c4c32258a7..4553aa4abd2b 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -39,18 +39,16 @@ object DataReadMethod extends Enumeration with Serializable { */ @DeveloperApi class InputMetrics private ( - val readMethod: DataReadMethod.Value, _bytesRead: Accumulator[Long], - _recordsRead: Accumulator[Long]) + _recordsRead: Accumulator[Long], + _readMethod: Accumulator[String]) extends Serializable { - private[executor] def this( - readMethod: DataReadMethod.Value, - accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - readMethod, - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.input.RECORDS_READ)) + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.BYTES_READ), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.RECORDS_READ), + TaskMetrics.getAccum[String](accumMap, InternalAccumulator.input.READ_METHOD)) } /** @@ -62,10 +60,8 @@ class InputMetrics private ( * * A better alternative to use is [[TaskMetrics.registerInputMetrics]]. */ - private[spark] def this(readMethod: DataReadMethod.Value) { - this( - readMethod, - InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) + private[executor] def this() { + this(InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) } /** @@ -78,6 +74,13 @@ class InputMetrics private ( */ def recordsRead: Long = _recordsRead.localValue + /** + * The source from which this task reads its input. + */ + def readMethod: DataReadMethod.Value = DataReadMethod.withName(_readMethod.localValue) + private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = + _readMethod.setValue(v.toString) } diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index e6318f7388d1..7f26a29d8971 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -38,18 +38,16 @@ object DataWriteMethod extends Enumeration with Serializable { */ @DeveloperApi class OutputMetrics private ( - val writeMethod: DataWriteMethod.Value, _bytesWritten: Accumulator[Long], - _recordsWritten: Accumulator[Long]) + _recordsWritten: Accumulator[Long], + _writeMethod: Accumulator[String]) extends Serializable { - private[executor] def this( - writeMethod: DataWriteMethod.Value, - accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - writeMethod, - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.BYTES_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.output.RECORDS_WRITTEN)) + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.BYTES_WRITTEN), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.RECORDS_WRITTEN), + TaskMetrics.getAccum[String](accumMap, InternalAccumulator.output.WRITE_METHOD)) } /** @@ -62,6 +60,13 @@ class OutputMetrics private ( */ def recordsWritten: Long = _recordsWritten.localValue + /** + * The source to which this task writes its input. + */ + def writeMethod: DataWriteMethod.Value = DataWriteMethod.withName(_writeMethod.localValue) + private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) + private[spark] def setWriteMethod(v: DataWriteMethod.Value): Unit = + _writeMethod.setValue(v.toString) } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 64f293e982bf..4ef854feca5d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -37,12 +37,12 @@ class ShuffleReadMetrics private ( private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) } /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 0c30bc938d61..0395fd97a5ef 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -34,9 +34,9 @@ class ShuffleWriteMetrics private ( private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { this( - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), - TaskMetrics.getLongAccum(accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), + TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) } /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 742b165a7f50..7c9ecbae4add 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -21,7 +21,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} @@ -44,7 +43,7 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * Additional accumulators registered here have no such requirements. */ @DeveloperApi -class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends Serializable { +class TaskMetrics private[spark](initialAccums: Seq[Accumulable[_, _]]) extends Serializable { import InternalAccumulator._ @@ -76,14 +75,14 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends s"accumulators passed to TaskMetrics:\n ${initialAccums.map(_.name.get).mkString("\n")}") // Each metric is internally represented as an accumulator - private val _executorDeserializeTime = getLongAccum(EXECUTOR_DESERIALIZE_TIME) - private val _executorRunTime = getLongAccum(EXECUTOR_RUN_TIME) - private val _resultSize = getLongAccum(RESULT_SIZE) - private val _jvmGCTime = getLongAccum(JVM_GC_TIME) - private val _resultSerializationTime = getLongAccum(RESULT_SERIALIZATION_TIME) - private val _memoryBytesSpilled = getLongAccum(MEMORY_BYTES_SPILLED) - private val _diskBytesSpilled = getLongAccum(DISK_BYTES_SPILLED) - private val _peakExecutionMemory = getLongAccum(PEAK_EXECUTION_MEMORY) + private val _executorDeserializeTime = getAccum[Long](EXECUTOR_DESERIALIZE_TIME) + private val _executorRunTime = getAccum[Long](EXECUTOR_RUN_TIME) + private val _resultSize = getAccum[Long](RESULT_SIZE) + private val _jvmGCTime = getAccum[Long](JVM_GC_TIME) + private val _resultSerializationTime = getAccum[Long](RESULT_SERIALIZATION_TIME) + private val _memoryBytesSpilled = getAccum[Long](MEMORY_BYTES_SPILLED) + private val _diskBytesSpilled = getAccum[Long](DISK_BYTES_SPILLED) + private val _peakExecutionMemory = getAccum[Long](PEAK_EXECUTION_MEMORY) /** * Time taken on the executor to deserialize this task. @@ -137,38 +136,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends private[spark] def incDiskBytesSpilled(v: Long) = _diskBytesSpilled.add(v) private[spark] def incPeakExecutionMemory(v: Long) = _peakExecutionMemory.add(v) - /** - * Register an accumulator with this task so we can access its value in [[accumulatorUpdates]]. - */ - private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit = { - accums += a - } - - /** - * Return all the accumulators used on this task. Note: This is not a copy. - */ - private[spark] def accumulators: Seq[Accumulable[_, _]] = accums - - /** - * Get a Long accumulator from the given map by name, assuming it exists. - * Note: this only searches the initial set passed into the constructor. - */ - private[spark] def getLongAccum(name: String): Accumulator[Long] = { - TaskMetrics.getLongAccum(initialAccumsMap, name) - } - - /** - * Return a map from accumulator ID to the accumulator's latest value in this task. - */ - def accumulatorUpdates(): Map[Long, Any] = accums.map { a => (a.id, a.localValue) }.toMap - - - /** - * Storage statuses of any blocks that have been updated as a result of this task. - */ - // TODO: make me an accumulator; right now this doesn't get sent to the driver. - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - /* ============================ * | OUTPUT METRICS | @@ -187,7 +154,8 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends */ def registerOutputMetrics(writeMethod: DataWriteMethod.Value): OutputMetrics = synchronized { _outputMetrics.getOrElse { - val metrics = new OutputMetrics(writeMethod, initialAccumsMap.toMap) + val metrics = new OutputMetrics(initialAccumsMap.toMap) + metrics.setWriteMethod(writeMethod) _outputMetrics = Some(metrics) metrics } @@ -209,10 +177,11 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends /** * Get or create a new [[InputMetrics]] associated with this task. */ - private[spark] def registerInputMetrics(readMethod: DataReadMethod): InputMetrics = { + private[spark] def registerInputMetrics(readMethod: DataReadMethod.Value): InputMetrics = { synchronized { val metrics = _inputMetrics.getOrElse { - val metrics = new InputMetrics(readMethod, initialAccumsMap.toMap) + val metrics = new InputMetrics(initialAccumsMap.toMap) + metrics.setReadMethod(readMethod) _inputMetrics = Some(metrics) metrics } @@ -224,7 +193,9 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends if (metrics.readMethod == readMethod) { metrics } else { - new InputMetrics(readMethod) + val m = new InputMetrics + m.setReadMethod(readMethod) + m } } } @@ -304,6 +275,66 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulable[_, _]]) extends agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) } + + /* ========================== * + | OTHER THINGS | + * ========================== */ + + /** + * Register an accumulator with this task so we can access its value in [[accumulatorUpdates]]. + */ + private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit = { + accums += a + } + + /** + * Return all the accumulators used on this task. Note: This is not a copy. + */ + private[spark] def accumulators: Seq[Accumulable[_, _]] = accums + + /** + * Get a Long accumulator from the given map by name, assuming it exists. + * Note: this only searches the initial set passed into the constructor. + */ + private[spark] def getAccum[Long](name: String): Accumulator[Long] = { + TaskMetrics.getAccum[Long](initialAccumsMap, name) + } + + /** + * Return a map from accumulator ID to the accumulator's latest value in this task. + */ + def accumulatorUpdates(): Map[Long, Any] = accums.map { a => (a.id, a.localValue) }.toMap + + + /** + * Storage statuses of any blocks that have been updated as a result of this task. + */ + // TODO: make me an accumulator; right now this doesn't get sent to the driver. + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + + /** + * Return whether some accumulators with the given prefix have already been set. + * This only considers the initial set passed into the constructor. + */ + private def accumsAlreadySet(prefix: String): Boolean = { + initialAccumsMap.filterKeys(_.startsWith(prefix)).values.exists { a => a.localValue != a.zero } + } + + // If we are reconstructing this TaskMetrics on the driver, some metrics may already be set. + // If so, initialize all relevant metrics classes so listeners can access them downstream. + if (accumsAlreadySet(SHUFFLE_READ_METRICS_PREFIX)) { + _shuffleReadMetrics = Some(new ShuffleReadMetrics(initialAccumsMap)) + } + if (accumsAlreadySet(SHUFFLE_WRITE_METRICS_PREFIX)) { + _shuffleWriteMetrics = Some(new ShuffleWriteMetrics(initialAccumsMap)) + } + if (accumsAlreadySet(OUTPUT_METRICS_PREFIX)) { + _outputMetrics = Some(new OutputMetrics(initialAccumsMap)) + } + if (accumsAlreadySet(INPUT_METRICS_PREFIX)) { + _inputMetrics = Some(new InputMetrics(initialAccumsMap)) + } + } @@ -314,13 +345,13 @@ private[spark] object TaskMetrics { /** * Get a Long accumulator from the given map by name, assuming it exists. */ - def getLongAccum( + def getAccum[T]( accumMap: Map[String, Accumulable[_, _]], - name: String): Accumulator[Long] = { + name: String): Accumulator[T] = { assert(accumMap.contains(name), s"metric '$name' is missing") try { // Note: we can't do pattern matching here because types are erased by compile time - accumMap(name).asInstanceOf[Accumulator[Long]] + accumMap(name).asInstanceOf[Accumulator[T]] } catch { case _: ClassCastException => throw new SparkException(s"attempted to access invalid accumulator $name as a long metric") diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 6590cf6ffd24..074f159ab7a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -45,8 +45,8 @@ private[spark] class ResultTask[T, U]( partition: Partition, locs: Seq[TaskLocation], val outputId: Int, - internalAccumulators: Seq[Accumulator[Long]]) - extends Task[U](stageId, stageAttemptId, partition.index, internalAccumulators) + override val initialAccumulators: Seq[Accumulator[_]]) + extends Task[U](stageId, stageAttemptId, partition.index, initialAccumulators) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index ea97ef0e746d..c32ba314d73d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -44,8 +44,8 @@ private[spark] class ShuffleMapTask( taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient private var locs: Seq[TaskLocation], - internalAccumulators: Seq[Accumulator[Long]]) - extends Task[MapStatus](stageId, stageAttemptId, partition.index, internalAccumulators) + override val initialAccumulators: Seq[Accumulator[_]]) + extends Task[MapStatus](stageId, stageAttemptId, partition.index, initialAccumulators) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 7ea24a217bd3..c1c8b47128f2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -74,10 +74,10 @@ private[scheduler] abstract class Stage( val name: String = callSite.shortForm val details: String = callSite.longForm - private var _internalAccumulators: Seq[Accumulator[Long]] = Seq.empty + private var _internalAccumulators: Seq[Accumulator[_]] = Seq.empty /** Internal accumulators shared across all tasks in this stage. */ - def internalAccumulators: Seq[Accumulator[Long]] = _internalAccumulators + def internalAccumulators: Seq[Accumulator[_]] = _internalAccumulators /** * Re-initialize the internal accumulators associated with this stage. diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 9b8015e9493f..07e4585ac765 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -47,7 +47,7 @@ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - val initialAccumulators: Seq[Accumulator[Long]]) extends Serializable { + val initialAccumulators: Seq[Accumulator[_]]) extends Serializable { /** * Called by [[Executor]] to run this task. diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index f8f825ae9890..9bd603609bb9 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -166,7 +166,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val accumulatorValues = taskContext.taskMetrics.accumulatorUpdates() assert(accumulators.size > 0) assert(accumulators.forall(_.isInternal)) - val testAccum = taskContext.taskMetrics.getLongAccum(TEST_ACCUM) + val testAccum = taskContext.taskMetrics.getAccum[Long](TEST_ACCUM) assert(accumulatorValues.size === accumulators.size) assert(accumulatorValues.contains(testAccum.id)) } @@ -178,7 +178,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -213,17 +213,17 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 10 + TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().taskMetrics().getLongAccum(TEST_ACCUM) += 100 + TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -271,7 +271,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => val taskContext = TaskContext.get() - taskContext.taskMetrics.getLongAccum(TEST_ACCUM) += 1 + taskContext.taskMetrics.getAccum[Long](TEST_ACCUM) += 1 // Fail the first attempts of a subset of the tasks if (failCondition(i) && taskContext.attemptNumber() == 0) { throw new Exception("Failing a task intentionally.") From c029f62be0628af904fe1a1ad795e8208f55e51d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 15:52:02 -0800 Subject: [PATCH 27/65] Fix *ShuffleSuite This fixes a bug where when we reconstruct TaskMetrics we just pass in mutable accumulators, such that when new tasks come in they change the values of the old tasks. A more subtle bug here is that we were passing in the accumulated values instead of the local task values. Both are now fixed. TODO: write a test for all of these please. --- .../scala/org/apache/spark/Accumulators.scala | 20 ++++++- .../apache/spark/executor/InputMetrics.scala | 4 +- .../apache/spark/executor/OutputMetrics.scala | 4 +- .../spark/executor/ShuffleReadMetrics.scala | 4 +- .../spark/executor/ShuffleWriteMetrics.scala | 4 +- .../apache/spark/executor/TaskMetrics.scala | 33 ++++++------ .../apache/spark/scheduler/DAGScheduler.scala | 53 +++++++++++-------- .../org/apache/spark/AccumulatorSuite.scala | 12 ++--- 8 files changed, 80 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index ad0abd1ac3cb..9fb886d71726 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -92,6 +92,15 @@ class Accumulable[R, T] private[spark] ( */ private[spark] def isInternal: Boolean = internal + /** + * Return a copy of this [[Accumulable]] without its current value. + */ + private[spark] def copy(newValue: Any): Accumulable[R, T] = { + val a = new Accumulable[R, T](initialValue, param, name, internal, countFailedValues) + a.setValue(newValue.asInstanceOf[R]) + a + } + /** * Add more data to this accumulator / accumulable * @param term the data to add @@ -281,6 +290,15 @@ class Accumulator[T] private[spark] ( override val countFailedValues: Boolean = false) extends Accumulable[T, T](initialValue, param, name, internal, countFailedValues) { + /** + * Return a copy of this [[Accumulator]] without its current value. + */ + private[spark] override def copy(newValue: Any): Accumulator[T] = { + val a = new Accumulator[T](initialValue, param, name, internal, countFailedValues) + a.setValue(newValue.asInstanceOf[T]) + a + } + def this(initialValue: T, param: AccumulatorParam[T], name: Option[String]) = { this(initialValue, param, name, false) } @@ -398,7 +416,7 @@ private[spark] object Accumulators extends Logging { private[spark] object InternalAccumulator { // Prefixes used in names of internal task level metrics - val METRICS_PREFIX = "metrics." + private val METRICS_PREFIX = "metrics." val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." val OUTPUT_METRICS_PREFIX = METRICS_PREFIX + "output." diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 4553aa4abd2b..fd530e2e0d2b 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.{Accumulator, InternalAccumulator} import org.apache.spark.annotation.DeveloperApi @@ -44,7 +44,7 @@ class InputMetrics private ( _readMethod: Accumulator[String]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulator[_]]) { this( TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.BYTES_READ), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.RECORDS_READ), diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index 7f26a29d8971..36639abb6ba4 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.{Accumulator, InternalAccumulator} import org.apache.spark.annotation.DeveloperApi @@ -43,7 +43,7 @@ class OutputMetrics private ( _writeMethod: Accumulator[String]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulator[_]]) { this( TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.BYTES_WRITTEN), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.RECORDS_WRITTEN), diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 4ef854feca5d..6a308a8d7d65 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.{Accumulator, InternalAccumulator} import org.apache.spark.annotation.DeveloperApi @@ -35,7 +35,7 @@ class ShuffleReadMetrics private ( _recordsRead: Accumulator[Long]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulator[_]]) { this( TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 0395fd97a5ef..8d86868f64d0 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator} +import org.apache.spark.{Accumulator, InternalAccumulator} import org.apache.spark.annotation.DeveloperApi @@ -32,7 +32,7 @@ class ShuffleWriteMetrics private ( _shuffleWriteTime: Accumulator[Long]) extends Serializable { - private[executor] def this(accumMap: Map[String, Accumulable[_, _]]) { + private[executor] def this(accumMap: Map[String, Accumulator[_]]) { this( TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 7c9ecbae4add..e72574c2a293 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -43,7 +43,7 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * Additional accumulators registered here have no such requirements. */ @DeveloperApi -class TaskMetrics private[spark](initialAccums: Seq[Accumulable[_, _]]) extends Serializable { +class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Serializable { import InternalAccumulator._ @@ -61,7 +61,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulable[_, _]]) extends /** * A map for quickly accessing the initial set of accumulators by name. */ - private val initialAccumsMap: Map[String, Accumulable[_, _]] = { + private val initialAccumsMap: Map[String, Accumulator[_]] = { initialAccums.map { a => assert(a.name.isDefined, "initial accumulators passed to TaskMetrics should be named") val name = a.name.get @@ -75,14 +75,14 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulable[_, _]]) extends s"accumulators passed to TaskMetrics:\n ${initialAccums.map(_.name.get).mkString("\n")}") // Each metric is internally represented as an accumulator - private val _executorDeserializeTime = getAccum[Long](EXECUTOR_DESERIALIZE_TIME) - private val _executorRunTime = getAccum[Long](EXECUTOR_RUN_TIME) - private val _resultSize = getAccum[Long](RESULT_SIZE) - private val _jvmGCTime = getAccum[Long](JVM_GC_TIME) - private val _resultSerializationTime = getAccum[Long](RESULT_SERIALIZATION_TIME) - private val _memoryBytesSpilled = getAccum[Long](MEMORY_BYTES_SPILLED) - private val _diskBytesSpilled = getAccum[Long](DISK_BYTES_SPILLED) - private val _peakExecutionMemory = getAccum[Long](PEAK_EXECUTION_MEMORY) + private val _executorDeserializeTime = getAccum(EXECUTOR_DESERIALIZE_TIME) + private val _executorRunTime = getAccum(EXECUTOR_RUN_TIME) + private val _resultSize = getAccum(RESULT_SIZE) + private val _jvmGCTime = getAccum(JVM_GC_TIME) + private val _resultSerializationTime = getAccum(RESULT_SERIALIZATION_TIME) + private val _memoryBytesSpilled = getAccum(MEMORY_BYTES_SPILLED) + private val _diskBytesSpilled = getAccum(DISK_BYTES_SPILLED) + private val _peakExecutionMemory = getAccum(PEAK_EXECUTION_MEMORY) /** * Time taken on the executor to deserialize this task. @@ -296,7 +296,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulable[_, _]]) extends * Get a Long accumulator from the given map by name, assuming it exists. * Note: this only searches the initial set passed into the constructor. */ - private[spark] def getAccum[Long](name: String): Accumulator[Long] = { + private[spark] def getAccum(name: String): Accumulator[Long] = { TaskMetrics.getAccum[Long](initialAccumsMap, name) } @@ -343,18 +343,19 @@ private[spark] object TaskMetrics { def empty: TaskMetrics = new TaskMetrics /** - * Get a Long accumulator from the given map by name, assuming it exists. + * Get an accumulator from the given map by name, assuming it exists. */ def getAccum[T]( - accumMap: Map[String, Accumulable[_, _]], + accumMap: Map[String, Accumulator[_]], name: String): Accumulator[T] = { assert(accumMap.contains(name), s"metric '$name' is missing") + val accum = accumMap(name) try { // Note: we can't do pattern matching here because types are erased by compile time - accumMap(name).asInstanceOf[Accumulator[T]] + accum.asInstanceOf[Accumulator[T]] } catch { - case _: ClassCastException => - throw new SparkException(s"attempted to access invalid accumulator $name as a long metric") + case e: ClassCastException => + throw new SparkException(s"accumulator $name was of unexpected type", e) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d58c703665be..46f4c76e1106 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1116,37 +1116,44 @@ class DAGScheduler( * * Note: If the task failed, we may return null after attempting to reconstruct the * [[TaskMetrics]] in vain. + * + * TODO: write tests here. */ private def reconstructTaskMetrics(task: Task[_], accumUpdates: Map[Long, Any]): TaskMetrics = { if (accumUpdates == null) { return null } - // We created the internal accumulators on the driver and expect the executor to send back - // accumulator values with matching IDs. If there are missing values, we cannot reconstruct - // the original TaskMetrics, so just return null. This might happen if the task failed. - val missingAccums = task.initialAccumulators.filter { a => !accumUpdates.contains(a.id) } - if (missingAccums.nonEmpty) { - val missingValuesString = missingAccums.map(_.name.get).mkString("[", ",", "]") - logWarning(s"Not reconstructing metrics for task ${task.partitionId} because its " + - s"accumulator updates had missing values: $missingValuesString. This could happen " + - s"if the task failed.") - return null - } - // TODO: the shuffle metrics and stuff are currently not set. This is failing tests. - val metrics = new TaskMetrics(task.initialAccumulators) - val registeredAccumIds = task.initialAccumulators.map(_.id).toSet - // Register all remaining accumulators separately because these may not be named. - accumUpdates - .flatMap { case (id, _) => - Accumulators.getAccum(id).orElse { - logWarning(s"Task ${task.partitionId} returned accumulator $id that was " + - s"not registered on the driver.") + val taskId = task.partitionId + try { + // Initial accumulators are passed into the TaskMetrics constructor first because these + // are required to be uniquely named. The rest of the accumulators from this task are + // registered later because they need not satisfy this requirement. + val initialAccumsMap = + task.initialAccumulators.map { a => (a.id, a) }.toMap[Long, Accumulator[_]] + val (initialAccumUpdates, otherAccumUpdates) = + accumUpdates.partition { case (id, _) => initialAccumsMap.contains(id) } + // Note: it is important that we copy the accumulators here since they are used across + // many tasks and we want to maintain a snapshot of their local task values when we post + // them to listeners downstream. Otherwise, when a new task comes in the listener may get + // a different value for an old task. + val newInitialAccums = initialAccumUpdates.map { case (id, taskValue) => + initialAccumsMap(id).copy(taskValue) + } + val otherAccums = otherAccumUpdates.flatMap { case (id, taskValue) => + Accumulators.getAccum(id).map(_.copy(taskValue)).orElse { + logWarning(s"Task $taskId returned unregistered accumulator $id.") None } } - .filter { a => !registeredAccumIds.contains(a.id) } - .foreach(metrics.registerAccumulator) - metrics + val metrics = new TaskMetrics(newInitialAccums.toSeq) + otherAccums.foreach(metrics.registerAccumulator) + metrics + } catch { + // Do not crash the scheduler if reconstruction fails + case NonFatal(e) => + logError(s"Error when attempting to reconstruct metrics for task $taskId", e) + null + } } /** diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 9bd603609bb9..8aa38cda29ec 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -166,7 +166,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val accumulatorValues = taskContext.taskMetrics.accumulatorUpdates() assert(accumulators.size > 0) assert(accumulators.forall(_.isInternal)) - val testAccum = taskContext.taskMetrics.getAccum[Long](TEST_ACCUM) + val testAccum = taskContext.taskMetrics.getAccum(TEST_ACCUM) assert(accumulatorValues.size === accumulators.size) assert(accumulatorValues.contains(testAccum.id)) } @@ -178,7 +178,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -213,17 +213,17 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 10 + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum[Long](TEST_ACCUM) += 100 + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -271,7 +271,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => val taskContext = TaskContext.get() - taskContext.taskMetrics.getAccum[Long](TEST_ACCUM) += 1 + taskContext.taskMetrics.getAccum(TEST_ACCUM) += 1 // Fail the first attempts of a subset of the tasks if (failCondition(i) && taskContext.attemptNumber() == 0) { throw new Exception("Failing a task intentionally.") From b3c51dd3930f8c304f0fc67fc476d95d066f9a1e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 17:45:48 -0800 Subject: [PATCH 28/65] Fix DAGSchedulerSuite The fake accumulator values should no longer all be Longs. Ugh. --- .../scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala | 1 - .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index e85516aca493..fa3ce0eca159 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,6 @@ import scala.collection.Map import scala.language.existentials import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite 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 f1dd7ce8e696..928a98a127c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1922,7 +1922,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou result: Any, extraAccumUpdates: Map[Long, Any] = Map[Long, Any](), taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { - val accumUpdates = task.initialAccumulators.map { a => (a.id, 0L) }.toMap ++ extraAccumUpdates + val accumUpdates = + task.initialAccumulators.map { a => (a.id, a.zero) }.toMap ++ extraAccumUpdates CompletionEvent(task, reason, result, accumUpdates, taskInfo) } From d531f3fff759dbfca6f24eb41ec37428ea055785 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 8 Jan 2016 22:51:17 -0800 Subject: [PATCH 29/65] Simplify accumulator update code a little --- .../scala/org/apache/spark/Accumulators.scala | 15 ++--------- .../apache/spark/scheduler/DAGScheduler.scala | 25 ++++++++----------- 2 files changed, 13 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 9fb886d71726..d358f091bae9 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -382,27 +382,16 @@ private[spark] object Accumulators extends Logging { } } - // Add values to the original accumulators with some given IDs - def add(values: Map[Long, Any]): Unit = synchronized { - for ((id, value) <- values) { - if (originals.contains(id)) { - getAccum(id).foreach { _.asInstanceOf[Accumulable[Any, Any]] ++= value } - } else { - logWarning(s"Ignoring accumulator update for unknown accumulator id $id") - } - } - } - /** * Return the accumulator registered with the given ID, if any. */ - def getAccum(id: Long): Option[Accumulable[_, _]] = { + def get(id: Long): Option[Accumulable[_, _]] = { originals.get(id).map { weakRef => // Since we are storing weak references, we must check whether the underlying data is valid. weakRef.get match { case Some(accum) => accum case None => - throw new IllegalAccessError("Attempted to access garbage collected Accumulator.") + throw new IllegalAccessError(s"Attempted to access garbage collected accumulator $id") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 46f4c76e1106..8b6202b27661 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1071,23 +1071,22 @@ class DAGScheduler( } } - /** Merge updates from a task to our local accumulator values */ + /** + * Update accumulators + */ private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task val stage = stageIdToStage(task.stageId) if (event.accumUpdates != null) { try { - Accumulators.add(event.accumUpdates) - event.accumUpdates.foreach { case (id, partialValue) => - // In this instance, although the reference in Accumulators.originals is a WeakRef, - // it's guaranteed to exist since the event.accumUpdates Map exists - - val acc = Accumulators.originals(id).get match { + // Find the previously registered accumulator and update it + val acc: Accumulable[Any, Any] = Accumulators.get(id) match { case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] - case None => throw new NullPointerException("Non-existent reference to Accumulator") + case None => + throw new SparkException(s"attempted to access non-existent accumulator $id") } - + acc ++= partialValue // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && partialValue != acc.zero) { val name = acc.name.get @@ -1099,10 +1098,8 @@ class DAGScheduler( } } } catch { - // If we see an exception during accumulator update, just log the - // error and move on. - case e: Exception => - logError(s"Failed to update accumulators for $task", e) + case NonFatal(e) => + logError(s"Failed to update accumulators for ${task.partitionId}", e) } } } @@ -1140,7 +1137,7 @@ class DAGScheduler( initialAccumsMap(id).copy(taskValue) } val otherAccums = otherAccumUpdates.flatMap { case (id, taskValue) => - Accumulators.getAccum(id).map(_.copy(taskValue)).orElse { + Accumulators.get(id).map(_.copy(taskValue)).orElse { logWarning(s"Task $taskId returned unregistered accumulator $id.") None } From 8391ef81dfc6cb78a4a6f4bfe459e6569f5f7817 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 9 Jan 2016 00:24:10 -0800 Subject: [PATCH 30/65] Track updated blocks as an accumulator This should be the last thing left to port everything in TaskMetrics over to accumulators. TODO: add some tests. --- .../scala/org/apache/spark/Accumulators.scala | 37 ++++++++++++++----- .../scala/org/apache/spark/CacheManager.scala | 5 +-- .../apache/spark/executor/TaskMetrics.scala | 16 ++++---- .../spark/memory/StorageMemoryPool.scala | 6 +-- .../spark/storage/StorageStatusListener.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 27 +++++++------- .../org/apache/spark/CacheManagerSuite.scala | 2 +- .../storage/StorageStatusListenerSuite.scala | 12 +++--- .../spark/ui/storage/StorageTabSuite.scala | 8 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- 11 files changed, 68 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index d358f091bae9..bbb3c4d8f336 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -20,12 +20,12 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} import scala.collection.generic.Growable -import scala.collection.Map import scala.collection.mutable import scala.ref.WeakReference import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage.{BlockStatus, BlockId} import org.apache.spark.util.Utils /** @@ -352,6 +352,14 @@ object AccumulatorParam { def addInPlace(t1: String, t2: String): String = t1 + t2 def zero(initialValue: String): String = "" } + + // Note: this is expensive as it makes a copy of the list every time the caller adds an item. + // A better way to use this is to first accumulate the values yourself then them all at once. + private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { + def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 + def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] + } + } // TODO: The multi-thread support in accumulators is kind of lame; check @@ -404,6 +412,8 @@ private[spark] object Accumulators extends Logging { private[spark] object InternalAccumulator { + import AccumulatorParam._ + // Prefixes used in names of internal task level metrics private val METRICS_PREFIX = "metrics." val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." @@ -420,6 +430,7 @@ private[spark] object InternalAccumulator { val MEMORY_BYTES_SPILLED = METRICS_PREFIX + "memoryBytesSpilled" val DISK_BYTES_SPILLED = METRICS_PREFIX + "diskBytesSpilled" val PEAK_EXECUTION_MEMORY = METRICS_PREFIX + "peakExecutionMemory" + val UPDATED_BLOCK_STATUSES = METRICS_PREFIX + "updatedBlockStatuses" val TEST_ACCUM = METRICS_PREFIX + "testAccumulator" // Names of shuffle read metrics @@ -458,7 +469,6 @@ private[spark] object InternalAccumulator { * Note: this method does not register accumulators for cleanup. */ def create(): Seq[Accumulator[_]] = { - val maybeTestAccum = sys.props.get("spark.testing").map(_ => newLongMetric(TEST_ACCUM)).toSeq Seq[Accumulator[_]]( newLongMetric(EXECUTOR_DESERIALIZE_TIME), newLongMetric(EXECUTOR_RUN_TIME), @@ -467,12 +477,16 @@ private[spark] object InternalAccumulator { newLongMetric(RESULT_SERIALIZATION_TIME), newLongMetric(MEMORY_BYTES_SPILLED), newLongMetric(DISK_BYTES_SPILLED), - newLongMetric(PEAK_EXECUTION_MEMORY)) ++ + newLongMetric(PEAK_EXECUTION_MEMORY), + newMetric( + Seq.empty[(BlockId, BlockStatus)], + UPDATED_BLOCK_STATUSES, + new ListAccumulatorParam[(BlockId, BlockStatus)])) ++ createShuffleReadAccums() ++ createShuffleWriteAccums() ++ createInputAccums() ++ createOutputAccums() ++ - maybeTestAccum + sys.props.get("spark.testing").map(_ => newLongMetric(TEST_ACCUM)).toSeq } /** @@ -537,20 +551,25 @@ private[spark] object InternalAccumulator { accums } + /** + * Create a new accumulator representing an internal task metric. + */ + private def newMetric[T](initialValue: T, name: String, param: AccumulatorParam[T]) = { + new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) + } /** - * Create a new internal Long accumulator with the specified name. + * Create a new Long accumulator representing an internal task metric. */ private def newLongMetric(name: String): Accumulator[Long] = { - new Accumulator[Long](0L, AccumulatorParam.LongAccumulatorParam, Some(name), internal = true) + newMetric[Long](0L, name, LongAccumulatorParam) } /** - * Create a new internal String accumulator with the specified name. + * Create a new String accumulator representing an internal task metric. */ private def newStringMetric(name: String): Accumulator[String] = { - new Accumulator[String]( - "", AccumulatorParam.StringAccumulatorParam, Some(name), internal = true) + newMetric[String]("", name, StringAccumulatorParam) } } diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index a4d129a6e132..2139ce3b7e06 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -76,11 +76,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // Otherwise, cache the values and keep track of any updates in block statuses val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) - val metrics = context.taskMetrics - val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - metrics.updatedBlocks = Some(lastUpdatedBlocks ++ updatedBlocks.toSeq) + context.taskMetrics().incUpdatedBlocks(updatedBlocks) new InterruptibleIterator(context, cachedValues) - } finally { loading.synchronized { loading.remove(key) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e72574c2a293..7a07f1cc95df 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -83,6 +83,8 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser private val _memoryBytesSpilled = getAccum(MEMORY_BYTES_SPILLED) private val _diskBytesSpilled = getAccum(DISK_BYTES_SPILLED) private val _peakExecutionMemory = getAccum(PEAK_EXECUTION_MEMORY) + private val _updatedBlocks = + TaskMetrics.getAccum[Seq[(BlockId, BlockStatus)]](initialAccumsMap, UPDATED_BLOCK_STATUSES) /** * Time taken on the executor to deserialize this task. @@ -127,6 +129,11 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser */ def peakExecutionMemory: Long = _peakExecutionMemory.localValue + /** + * Storage statuses of any blocks that have been updated as a result of this task. + */ + def updatedBlocks: Seq[(BlockId, BlockStatus)] = _updatedBlocks.localValue + private[spark] def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) private[spark] def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) private[spark] def setResultSize(v: Long) = _resultSize.setValue(v) @@ -135,6 +142,8 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser private[spark] def incMemoryBytesSpilled(v: Long) = _memoryBytesSpilled.add(v) private[spark] def incDiskBytesSpilled(v: Long) = _diskBytesSpilled.add(v) private[spark] def incPeakExecutionMemory(v: Long) = _peakExecutionMemory.add(v) + private[spark] def incUpdatedBlocks(v: Seq[(BlockId, BlockStatus)]) = _updatedBlocks.add(v) + private[spark] def setUpdatedBlocks(v: Seq[(BlockId, BlockStatus)]) = _updatedBlocks.setValue(v) /* ============================ * @@ -305,13 +314,6 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser */ def accumulatorUpdates(): Map[Long, Any] = accums.map { a => (a.id, a.localValue) }.toMap - - /** - * Storage statuses of any blocks that have been updated as a result of this task. - */ - // TODO: make me an accumulator; right now this doesn't get sent to the driver. - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - /** * Return whether some accumulators with the given prefix have already been set. * This only considers the initial set passed into the constructor. diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 4036484aada2..98f67e5631d5 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -88,11 +88,7 @@ private[memory] class StorageMemoryPool(lock: Object) extends MemoryPool(lock) w if (numBytesToFree > 0) { memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, evictedBlocks) // Register evicted blocks, if any, with the active task metrics - Option(TaskContext.get()).foreach { tc => - val metrics = tc.taskMetrics() - val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) - } + Option(TaskContext.get()).foreach(_.taskMetrics().incUpdatedBlocks(evictedBlocks)) } // NOTE: If the memory store evicts blocks, then those evictions will synchronously call // back into this StorageMemoryPool in order to free memory. Therefore, these variables diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index ec711480ebf3..ee73b04ed630 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -63,7 +63,7 @@ class StorageStatusListener extends SparkListener { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { - val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) + val updatedBlocks = metrics.updatedBlocks if (updatedBlocks.length > 0) { updateStorageStatus(info.executorId, updatedBlocks) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 2d9b885c684b..126fbee67a97 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -63,8 +63,8 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val metrics = taskEnd.taskMetrics - if (metrics != null && metrics.updatedBlocks.isDefined) { - updateRDDInfo(metrics.updatedBlocks.get) + if (metrics != null && metrics.updatedBlocks.nonEmpty) { + updateRDDInfo(metrics.updatedBlocks) } } 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 f69721a9ffdd..d05e55a17182 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -320,12 +320,10 @@ private[spark] object JsonProtocol { ("Records Written" -> om.recordsWritten) }.getOrElse(JNothing) val updatedBlocks = - taskMetrics.updatedBlocks.map { blocks => - JArray(blocks.toList.map { case (id, status) => - ("Block ID" -> id.toString) ~ - ("Status" -> blockStatusToJson(status)) - }) - }.getOrElse(JNothing) + JArray(taskMetrics.updatedBlocks.toList.map { case (id, status) => + ("Block ID" -> id.toString) ~ + ("Status" -> blockStatusToJson(status)) + }) ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ ("Executor Run Time" -> taskMetrics.executorRunTime) ~ ("Result Size" -> taskMetrics.resultSize) ~ @@ -751,14 +749,15 @@ private[spark] object JsonProtocol { inputMetrics.incRecordsRead((inJson \ "Records Read").extractOpt[Long].getOrElse(0L)) } - metrics.updatedBlocks = - Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) - } - } + // Updated blocks + Utils.jsonOption(json \ "Updated Blocks").foreach { blocksJson => + metrics.setUpdatedBlocks(blocksJson.extract[List[JValue]].map { blockJson => + val id = BlockId((blockJson \ "Block ID").extract[String]) + val status = blockStatusFromJson(blockJson \ "Status") + (id, status) + }) + } + metrics } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index a601966fb177..bd49e1e349e6 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -95,6 +95,6 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before cacheManager = sc.env.cacheManager val context = TaskContext.empty() cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) - assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) + assert(context.taskMetrics.updatedBlocks.size === 2) } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 1a199beb3558..9ea9b749c976 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -85,8 +85,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(Seq(block3)) + taskMetrics1.setUpdatedBlocks(Seq(block1, block2)) + taskMetrics2.setUpdatedBlocks(Seq(block3)) // Task end with new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) @@ -108,8 +108,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) - taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) + taskMetrics1.setUpdatedBlocks(Seq(droppedBlock1, droppedBlock3)) + taskMetrics2.setUpdatedBlocks(Seq(droppedBlock2, droppedBlock3)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) @@ -133,8 +133,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(Seq(block3)) + taskMetrics1.setUpdatedBlocks(Seq(block1, block2)) + taskMetrics2.setUpdatedBlocks(Seq(block3)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 4b838a8ab133..bbf3aa2ed185 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -127,7 +127,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Task end with a few new persisted blocks, some from the same RDD val metrics1 = new TaskMetrics - metrics1.updatedBlocks = Some(Seq( + metrics1.setUpdatedBlocks(Seq( (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L, 0L)), (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L, 0L)), (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), @@ -149,7 +149,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Task end with a few dropped blocks val metrics2 = new TaskMetrics - metrics2.updatedBlocks = Some(Seq( + metrics2.setUpdatedBlocks(Seq( (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L, 0L)), (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L, 0L)), (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist @@ -177,8 +177,8 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { val taskMetrics1 = new TaskMetrics val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L, 0L)) val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L, 0L)) - taskMetrics0.updatedBlocks = Some(Seq(block0)) - taskMetrics1.updatedBlocks = Some(Seq(block1)) + taskMetrics0.setUpdatedBlocks(Seq(block0)) + taskMetrics1.setUpdatedBlocks(Seq(block1)) bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener.rddInfoList.size === 0) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 81993385a032..0debcd8cea35 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -555,7 +555,7 @@ class JsonProtocolSuite extends SparkFunSuite { metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) assertOptionEquals( metrics1.inputMetrics, metrics2.inputMetrics, assertInputMetricsEquals) - assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) + assertBlocksEquals(metrics1.updatedBlocks, metrics2.updatedBlocks) } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { @@ -793,7 +793,7 @@ class JsonProtocolSuite extends SparkFunSuite { sw.incRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) } // Make at most 6 blocks - t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => + t.setUpdatedBlocks((1 to (e % 5 + 1)).map { i => (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i, c%i)) }.toSeq) t From 972336bee0e6429eb54a741993201a1be31bcb3c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 9 Jan 2016 01:15:51 -0800 Subject: [PATCH 31/65] Minor clean ups + docs --- .../scala/org/apache/spark/Accumulators.scala | 18 +++++++++-------- .../org/apache/spark/executor/Executor.scala | 1 + .../apache/spark/executor/InputMetrics.scala | 3 ++- .../apache/spark/executor/OutputMetrics.scala | 1 + .../spark/executor/ShuffleReadMetrics.scala | 3 ++- .../spark/executor/ShuffleWriteMetrics.scala | 3 ++- .../apache/spark/executor/TaskMetrics.scala | 20 +++++++++---------- .../apache/spark/scheduler/DAGScheduler.scala | 16 +++++++-------- .../spark/scheduler/TaskSetManager.scala | 17 +++++++--------- .../spark/ui/jobs/JobProgressListener.scala | 4 +++- 10 files changed, 46 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index bbb3c4d8f336..6814b8a3d8a2 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -39,9 +39,6 @@ import org.apache.spark.util.Utils * * Operations are not thread-safe. * - * Note: all internal accumulators used within a task must have unique names because we access - * them by name in [[org.apache.spark.executor.TaskMetrics]]. - * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` * @param name human-readable name for use in Spark's web UI @@ -80,7 +77,9 @@ class Accumulable[R, T] private[spark] ( val zero = param.zero(initialValue) // Zero value to be passed to executors private var deserialized = false - // Avoid leaking accumulators on executors + // In certain places we create accumulators on the executors. If we register them here then + // we will never clean them up because there's no context cleaner on the executors. E.g. we + // manually create ShuffleWriteMetrics, which is a collection of accumulators, in some places. if (isDriver) { Accumulators.register(this) } @@ -93,7 +92,7 @@ class Accumulable[R, T] private[spark] ( private[spark] def isInternal: Boolean = internal /** - * Return a copy of this [[Accumulable]] without its current value. + * Return a copy of this [[Accumulable]] with a new value. */ private[spark] def copy(newValue: Any): Accumulable[R, T] = { val a = new Accumulable[R, T](initialValue, param, name, internal, countFailedValues) @@ -168,8 +167,8 @@ class Accumulable[R, T] private[spark] ( private[spark] def setValue(newValue: R): Unit = { value_ = newValue } /** - * Whether we are on the driver or the executors. - * Note: in local mode, this will inevitably return true even if we're on the executor. + * Whether we are on the driver or on the executors. + * Note: in local mode, this will inevitably return true even when we're on the executors. */ private def isDriver: Boolean = { Option(SparkEnv.get).map(_.isDriver).getOrElse(true) @@ -291,7 +290,7 @@ class Accumulator[T] private[spark] ( extends Accumulable[T, T](initialValue, param, name, internal, countFailedValues) { /** - * Return a copy of this [[Accumulator]] without its current value. + * Return a copy of this [[Accumulator]] with a new value. */ private[spark] override def copy(newValue: Any): Accumulator[T] = { val a = new Accumulator[T](initialValue, param, name, internal, countFailedValues) @@ -404,6 +403,9 @@ private[spark] object Accumulators extends Logging { } } + /** + * Clear all registered accumulators; for testing only. + */ def clear(): Unit = synchronized { originals.clear() } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 23555d663c65..175c563cbbe3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -448,6 +448,7 @@ private[spark] class Executor( } } + // TODO: don't send TaskMetrics here; send accumulator updates. val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) try { val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index fd530e2e0d2b..6388d1c4fee7 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -58,7 +58,7 @@ class InputMetrics private ( * because we want to ignore metrics from a second read method. In the future, we should revisit * whether this is needed. * - * A better alternative to use is [[TaskMetrics.registerInputMetrics]]. + * A better alternative is [[TaskMetrics.registerInputMetrics]]. */ private[executor] def this() { this(InternalAccumulator.createInputAccums().map { a => (a.name.get, a) }.toMap) @@ -83,4 +83,5 @@ class InputMetrics private ( private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = _readMethod.setValue(v.toString) + } diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index 36639abb6ba4..c2be4ed105c3 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -69,4 +69,5 @@ class OutputMetrics private ( private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) private[spark] def setWriteMethod(v: DataWriteMethod.Value): Unit = _writeMethod.setValue(v.toString) + } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 6a308a8d7d65..5f81f63aab56 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -52,7 +52,7 @@ class ShuffleReadMetrics private ( * many places only to merge their values together later. In the future, we should revisit * whether this is needed. * - * A better alternative to use is [[TaskMetrics.registerTempShuffleReadMetrics]]. + * A better alternative is [[TaskMetrics.registerTempShuffleReadMetrics]]. */ private[spark] def this() { this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) @@ -113,4 +113,5 @@ class ShuffleReadMetrics private ( private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) + } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 8d86868f64d0..c012836882aa 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -46,7 +46,7 @@ class ShuffleWriteMetrics private ( * many places only to merge their values together later. In the future, we should revisit * whether this is needed. * - * A better alternative to use is [[TaskMetrics.registerShuffleWriteMetrics]]. + * A better alternative is [[TaskMetrics.registerShuffleWriteMetrics]]. */ private[spark] def this() { this(InternalAccumulator.createShuffleWriteAccums().map { a => (a.name.get, a) }.toMap) @@ -76,4 +76,5 @@ class ShuffleWriteMetrics private ( private[spark] def decRecordsWritten(v: Long): Unit = { _recordsWritten.setValue(recordsWritten - v) } + } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 7a07f1cc95df..62907f12163b 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -24,19 +24,19 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} +// TODO: make this and related classes private + /** * :: DeveloperApi :: * Metrics tracked during the execution of a task. * - * This class is used to house metrics both for in-progress and completed tasks. In executors, - * both the task thread and the heartbeat thread write to the TaskMetrics. The heartbeat thread - * reads it to send in-progress metrics, and the task thread reads it to send metrics along with - * the completed task. - * - * So, when adding new fields, take into consideration that the whole object can be serialized for - * shipping off at any time to consumers of the SparkListener interface. + * This class is wrapper around a collection of internal accumulators that represent metrics + * associated with a task. The local values of these accumulators are sent from the executor + * to the driver when the task completes. These values are then merged into the corresponding + * accumulator previously registered on the driver. * - * TODO: update this comment. + * These accumulator updates are also sent to the driver periodically (on executor heartbeat) + * or when the task failed with an exception. * * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. * Each accumulator in this initial set must be named and marked as internal. @@ -303,7 +303,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser /** * Get a Long accumulator from the given map by name, assuming it exists. - * Note: this only searches the initial set passed into the constructor. + * Note: this only searches the initial set of accumulators passed into the constructor. */ private[spark] def getAccum(name: String): Accumulator[Long] = { TaskMetrics.getAccum[Long](initialAccumsMap, name) @@ -316,7 +316,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser /** * Return whether some accumulators with the given prefix have already been set. - * This only considers the initial set passed into the constructor. + * This only considers the initial set of accumulators passed into the constructor. */ private def accumsAlreadySet(prefix: String): Boolean = { initialAccumsMap.filterKeys(_.startsWith(prefix)).values.exists { a => a.localValue != a.zero } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8b6202b27661..a6b701d7cc39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1072,7 +1072,8 @@ class DAGScheduler( } /** - * Update accumulators + * Merge local values from a task into the corresponding accumulator previously + * registered here on the driver. */ private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task @@ -1107,9 +1108,9 @@ class DAGScheduler( /** * Reconstruct [[TaskMetrics]] from accumulator updates. * - * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. - * However, we need the latter to post task end events to listeners, so we need to - * reconstruct the metrics here on the driver. + * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, + * we need the latter to post task end events to listeners, so we need to reconstruct the + * metrics here on the driver. * * Note: If the task failed, we may return null after attempting to reconstruct the * [[TaskMetrics]] in vain. @@ -1168,9 +1169,9 @@ class DAGScheduler( // (3) Post SparkListenerTaskEnd event // (4) Post SparkListenerStageCompleted / SparkListenerJobEnd event - // Update accumulator values based on updates from this task. - // Note: we must do this before reconstructing TaskMetrics, otherwise the TaskMetrics - // will not have updated accumulator values. This is needed for the SQL UI, for instance. + // Update accumulator values based on updates from this task. Note: we must do this before + // reconstructing TaskMetrics, otherwise the TaskMetrics will not have the updated metrics. + // This is needed for the SQL UI, for instance. if (stageIdToStage.contains(stageId)) { val stage = stageIdToStage(stageId) // We should should update registered accumulators if this task succeeded or failed with @@ -1201,7 +1202,6 @@ class DAGScheduler( event.taskInfo.attemptNumber, // this is a task attempt number event.reason) - // Post task end event listenerBus.post(SparkListenerTaskEnd( stageId, task.stageAttemptId, taskType, event.reason, event.taskInfo, taskMetrics)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index bfc9559d1567..6acdb6b15bb0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -615,7 +615,7 @@ private[spark] class TaskSetManager( info.markSuccessful() removeRunningTask(tid) - // Update result size metric in `result`, which is not yet set. + // Update result size metric in `result` updateResultSize(tid, result, resultSize) // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the @@ -645,10 +645,11 @@ private[spark] class TaskSetManager( /** * Set the task result size in the accumulator updates received from the executors. * - * Note: If we did this on the executors we would have to serialize the result again after - * updating the size, which is potentially expensive. Also, we would have to do something - * extra for indirect task results. It's better to just do all of this in one place on the - * driver. + * If we did this on the executors we would have to serialize the result again after updating + * the size, which is potentially expensive. Also, we would have to do something extra for + * indirect task results. It's better to just do all of this in one place on the driver. + * + * TODO: write a test. */ private def updateResultSize(tid: Long, result: DirectTaskResult[_], resultSize: Long): Unit = { val index = taskInfos(tid).index @@ -692,12 +693,8 @@ private[spark] class TaskSetManager( None case ef: ExceptionFailure => - // ExceptionFailure's might have accumulator updates - if (ef.accumulatorUpdates != null) { - accumUpdates = ef.accumulatorUpdates - } - + accumUpdates = ef.accumulatorUpdates if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 4a9f8b30525f..71493d569cad 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -417,8 +417,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage * aggregate metrics by calculating deltas between the currently recorded metrics and the new * metrics. + * + * TODO: no need to do this! Just use accumulators. :) */ - def updateAggregateMetrics( + private def updateAggregateMetrics( stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, From 476394a7a06ea9634c6a1857fabbb0f3a17a1fe7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 11 Jan 2016 12:56:29 -0800 Subject: [PATCH 32/65] Replace all accumulator updates with Seq[AccumulableInfo] This makes it more consistent everywhere and simplifies some code. E.g. the code where we set the result size is now moved back to TaskResultGetter because of the decrease in code dependency. This also allows us to rewrite the heartbeat events using AccumulableInfo so we don't have to send TaskMetrics directly. --- .../org/apache/spark/TaskEndReason.scala | 11 ++-- .../org/apache/spark/executor/Executor.scala | 6 +- .../apache/spark/executor/TaskMetrics.scala | 10 ++-- .../spark/scheduler/AccumulableInfo.scala | 57 ++++++------------- .../apache/spark/scheduler/DAGScheduler.scala | 35 +++++++----- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 11 ++-- .../apache/spark/scheduler/TaskResult.scala | 15 ++--- .../spark/scheduler/TaskResultGetter.scala | 19 ++++++- .../spark/scheduler/TaskSchedulerImpl.scala | 5 +- .../spark/scheduler/TaskSetManager.scala | 32 +---------- .../status/api/v1/AllStagesResource.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala | 17 +++--- .../org/apache/spark/AccumulatorSuite.scala | 32 +++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 20 ++++--- .../spark/scheduler/TaskSetManagerSuite.scala | 22 ++++--- .../org/apache/spark/ui/StagePageSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 5 +- .../spark/sql/execution/ui/SQLListener.scala | 15 +++-- .../sql/execution/ui/SQLListenerSuite.scala | 5 +- .../sql/util/DataFrameCallbackSuite.scala | 2 +- 21 files changed, 156 insertions(+), 173 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index d7d0a0611b4e..5363a93ef020 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -21,6 +21,7 @@ import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils @@ -116,7 +117,7 @@ case class ExceptionFailure( stackTrace: Array[StackTraceElement], fullStackTrace: String, exceptionWrapper: Option[ThrowableSerializationWrapper], - accumulatorUpdates: Map[Long, Any] = Map[Long, Any](), + accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], // always None, kept here for backward compatibility metrics: Option[TaskMetrics] = None) extends TaskFailedReason { @@ -128,14 +129,14 @@ case class ExceptionFailure( */ private[spark] def this( e: Throwable, - accumulatorUpdates: Map[Long, Any], + accumUpdates: Seq[AccumulableInfo], preserveCause: Boolean) { this(e.getClass.getName, e.getMessage, e.getStackTrace, Utils.exceptionString(e), - if (preserveCause) Some(new ThrowableSerializationWrapper(e)) else None, accumulatorUpdates) + if (preserveCause) Some(new ThrowableSerializationWrapper(e)) else None, accumUpdates) } - private[spark] def this(e: Throwable, accumulatorUpdates: Map[Long, Any]) { - this(e, accumulatorUpdates, preserveCause = true) + private[spark] def this(e: Throwable, accumUpdates: Seq[AccumulableInfo]) { + this(e, accumUpdates, preserveCause = true) } def exception: Option[Throwable] = exceptionWrapper.flatMap { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 175c563cbbe3..4ea2198ef149 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -31,7 +31,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rpc.RpcTimeout -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} +import org.apache.spark.scheduler.{AccumulableInfo, DirectTaskResult, IndirectTaskResult, Task} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ @@ -300,7 +300,7 @@ private[spark] class Executor( logError(s"Exception in $taskName (TID $taskId)", t) // Collect latest accumulator values to report back to the driver - val accumulatorUpdates: Map[Long, Any] = + val accumulatorUpdates: Seq[AccumulableInfo] = if (task != null) { task.metrics.foreach { m => m.setExecutorRunTime(System.currentTimeMillis() - taskStart) @@ -308,7 +308,7 @@ private[spark] class Executor( } task.collectAccumulatorUpdates(taskFailed = true) } else { - Map[Long, Any]() + Seq.empty[AccumulableInfo] } val serializedTaskEndReason = { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 62907f12163b..9fb760ff4315 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} @@ -296,11 +297,6 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser accums += a } - /** - * Return all the accumulators used on this task. Note: This is not a copy. - */ - private[spark] def accumulators: Seq[Accumulable[_, _]] = accums - /** * Get a Long accumulator from the given map by name, assuming it exists. * Note: this only searches the initial set of accumulators passed into the constructor. @@ -312,7 +308,9 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser /** * Return a map from accumulator ID to the accumulator's latest value in this task. */ - def accumulatorUpdates(): Map[Long, Any] = accums.map { a => (a.id, a.localValue) }.toMap + def accumulatorUpdates(): Seq[AccumulableInfo] = accums.map { a => + new AccumulableInfo(a.id, a.name.orNull, Some(a.localValue), None, a.isInternal) + } /** * Return whether some accumulators with the given prefix have already been set. diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 146cfb9ba803..e4315ffeeaa3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -22,44 +22,23 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. + * + * Note: once this is JSON serialized the types of `update` and `value` will be lost and be + * cast to strings. This is because the user can define an accumulator of any type and it will + * be difficult to preserve the type in consumers of the event log. + * + * @param id accumulator ID + * @param name accumulator name + * @param update partial value from a task, may be None if used on driver to describe a stage + * @param value total accumulated value so far, maybe None if used on executors to describe a task + * @param internal whether this accumulator was internal + * @param countFailedValues whether to count this accumulator's partial value if the task failed */ @DeveloperApi -class AccumulableInfo private[spark] ( - val id: Long, - val name: String, - val update: Option[String], // represents a partial update within a task - val value: String, - val internal: Boolean) { - - override def equals(other: Any): Boolean = other match { - case acc: AccumulableInfo => - this.id == acc.id && this.name == acc.name && - this.update == acc.update && this.value == acc.value && - this.internal == acc.internal - case _ => false - } - - override def hashCode(): Int = { - val state = Seq(id, name, update, value, internal) - state.map(_.hashCode).reduceLeft(31 * _ + _) - } -} - -object AccumulableInfo { - def apply( - id: Long, - name: String, - update: Option[String], - value: String, - internal: Boolean): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, internal) - } - - def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, internal = false) - } - - def apply(id: Long, name: String, value: String): AccumulableInfo = { - new AccumulableInfo(id, name, None, value, internal = false) - } -} +case class AccumulableInfo private[spark] ( + id: Long, + name: String, + update: Option[Any], + value: Option[Any], + internal: Boolean, + countFailedValues: Boolean = false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a6b701d7cc39..44d76bbccf9d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -208,7 +208,7 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], + accumUpdates: Seq[AccumulableInfo], taskInfo: TaskInfo): Unit = { eventProcessLoop.post(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } @@ -1078,9 +1078,14 @@ class DAGScheduler( private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task val stage = stageIdToStage(task.stageId) + var failedDuringThisOne: AccumulableInfo = null if (event.accumUpdates != null) { try { - event.accumUpdates.foreach { case (id, partialValue) => + event.accumUpdates.foreach { ainfo => + failedDuringThisOne = ainfo + val id = ainfo.id + assert(ainfo.update.isDefined, "accumulator from task should have a partial value") + val partialValue = ainfo.update.get // Find the previously registered accumulator and update it val acc: Accumulable[Any, Any] = Accumulators.get(id) match { case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] @@ -1091,16 +1096,15 @@ class DAGScheduler( // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && partialValue != acc.zero) { val name = acc.name.get - val value = s"${acc.value}" stage.latestInfo.accumulables(id) = - new AccumulableInfo(id, name, None, value, acc.isInternal) + new AccumulableInfo(id, name, None, Some(acc.value), acc.isInternal) event.taskInfo.accumulables += - new AccumulableInfo(id, name, Some(s"$partialValue"), value, acc.isInternal) + new AccumulableInfo(id, name, Some(partialValue), Some(acc.value), acc.isInternal) } } } catch { case NonFatal(e) => - logError(s"Failed to update accumulators for ${task.partitionId}", e) + logError(s"Failed to update accumulators for task ${task.partitionId}", e) } } } @@ -1117,7 +1121,9 @@ class DAGScheduler( * * TODO: write tests here. */ - private def reconstructTaskMetrics(task: Task[_], accumUpdates: Map[Long, Any]): TaskMetrics = { + private def reconstructTaskMetrics( + task: Task[_], + accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { if (accumUpdates == null) { return null } @@ -1128,17 +1134,20 @@ class DAGScheduler( // registered later because they need not satisfy this requirement. val initialAccumsMap = task.initialAccumulators.map { a => (a.id, a) }.toMap[Long, Accumulator[_]] - val (initialAccumUpdates, otherAccumUpdates) = - accumUpdates.partition { case (id, _) => initialAccumsMap.contains(id) } + val (initialAccumUpdates, otherAccumUpdates) = accumUpdates.partition { a => + assert(a.update.isDefined, "accumulator update from task should have a partial value") + initialAccumsMap.contains(a.id) + } // Note: it is important that we copy the accumulators here since they are used across // many tasks and we want to maintain a snapshot of their local task values when we post // them to listeners downstream. Otherwise, when a new task comes in the listener may get // a different value for an old task. - val newInitialAccums = initialAccumUpdates.map { case (id, taskValue) => - initialAccumsMap(id).copy(taskValue) + val newInitialAccums = initialAccumUpdates.map { a => + initialAccumsMap(a.id).copy(a.update.get) } - val otherAccums = otherAccumUpdates.flatMap { case (id, taskValue) => - Accumulators.get(id).map(_.copy(taskValue)).orElse { + val otherAccums = otherAccumUpdates.flatMap { a => + val id = a.id + Accumulators.get(id).map(_.copy(a.update.get)).orElse { logWarning(s"Task $taskId returned unregistered accumulator $id.") None } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index fa3ce0eca159..6b6bccf0b39f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -72,7 +72,7 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], + accumUpdates: Seq[AccumulableInfo], taskInfo: TaskInfo) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 07e4585ac765..5d74b0cfaf23 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} +import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer import scala.collection.mutable.HashMap @@ -126,14 +126,11 @@ private[spark] abstract class Task[T]( * Collect the latest values of accumulators used in this task. If the task failed, * filter out the accumulators whose values should not be included on failures. */ - def collectAccumulatorUpdates(taskFailed: Boolean = false): Map[Long, Any] = { + def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulableInfo] = { if (context != null) { - context.taskMetrics.accumulators - .filter { a => !taskFailed || a.countFailedValues } - .map { a => (a.id, a.localValue) } - .toMap + context.taskMetrics.accumulatorUpdates().filter { a => !taskFailed || a.countFailedValues } } else { - Map[Long, Any]() + Seq.empty[AccumulableInfo] } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index bc506cf2e768..03135e63d755 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -20,8 +20,7 @@ package org.apache.spark.scheduler import java.io._ import java.nio.ByteBuffer -import scala.collection.Map -import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkEnv import org.apache.spark.storage.BlockId @@ -37,7 +36,7 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] class DirectTaskResult[T]( var valueBytes: ByteBuffer, - var accumUpdates: Map[Long, Any]) + var accumUpdates: Seq[AccumulableInfo]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false @@ -48,12 +47,8 @@ private[spark] class DirectTaskResult[T]( override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(valueBytes.remaining) Utils.writeByteBuffer(valueBytes, out) - out.writeInt(accumUpdates.size) - for ((key, value) <- accumUpdates) { - out.writeLong(key) - out.writeObject(value) - } + accumUpdates.foreach(out.writeObject) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -66,9 +61,9 @@ private[spark] class DirectTaskResult[T]( if (numUpdates == 0) { accumUpdates = null } else { - val _accumUpdates = mutable.Map[Long, Any]() + val _accumUpdates = new ArrayBuffer[AccumulableInfo] for (i <- 0 until numUpdates) { - _accumUpdates(in.readLong()) = in.readObject() + _accumUpdates += in.readObject.asInstanceOf[AccumulableInfo] } accumUpdates = _accumUpdates } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index e9a745d18fc5..0509b1004903 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -45,7 +45,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } def enqueueSuccessfulTask( - taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { + taskSetManager: TaskSetManager, + tid: Long, + serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { try { @@ -82,7 +84,20 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul (deserializedResult, size) } - scheduler.handleSuccessfulTask(taskSetManager, tid, result, size.toLong) + // Set the task result size in the accumulator updates received from the executors. + // If we did this on the executors we would have to serialize the result again after + // updating the size, which is potentially expensive. TODO: write a test. + result.accumUpdates = result.accumUpdates.map { ainfo => + if (ainfo.name == InternalAccumulator.RESULT_SIZE) { + assert(ainfo.update.getOrElse(0L) == 0L, + "task result size should not have been set on the executors") + ainfo.copy(update = Some(size.toLong)) + } else { + ainfo + } + } + + scheduler.handleSuccessfulTask(taskSetManager, tid, result) } catch { case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index a8a4c58cc34d..6e3ef0e54f0f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -400,9 +400,8 @@ private[spark] class TaskSchedulerImpl( def handleSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, - taskResult: DirectTaskResult[_], - resultSize: Long): Unit = synchronized { - taskSetManager.handleSuccessfulTask(tid, taskResult, resultSize) + taskResult: DirectTaskResult[_]): Unit = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult) } def handleFailedTask( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 6acdb6b15bb0..7afffd6211ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -609,15 +609,12 @@ private[spark] class TaskSetManager( /** * Marks a task as successful and notifies the DAGScheduler that the task has ended. */ - def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_], resultSize: Long): Unit = { + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = { val info = taskInfos(tid) val index = info.index info.markSuccessful() removeRunningTask(tid) - // Update result size metric in `result` - updateResultSize(tid, result, resultSize) - // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not // "deserialize" the value when holding a lock to avoid blocking other threads. So we call @@ -642,29 +639,6 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } - /** - * Set the task result size in the accumulator updates received from the executors. - * - * If we did this on the executors we would have to serialize the result again after updating - * the size, which is potentially expensive. Also, we would have to do something extra for - * indirect task results. It's better to just do all of this in one place on the driver. - * - * TODO: write a test. - */ - private def updateResultSize(tid: Long, result: DirectTaskResult[_], resultSize: Long): Unit = { - val index = taskInfos(tid).index - val task = tasks(index) - val accumName = InternalAccumulator.RESULT_SIZE - val resultSizeAccum = task.initialAccumulators.find { a => a.name == Some(accumName) } - assert(resultSizeAccum.isDefined, s"did not find accumulator called '$accumName' in task") - val resultSizeAccumId = resultSizeAccum.get.id - assert(result.accumUpdates.contains(resultSizeAccumId), - s"did not find accumulator called '$accumName' in task result") - assert(result.accumUpdates(resultSizeAccumId) == 0L, - s"task result size '$accumName' should not have been set on the executors") - result.accumUpdates = result.accumUpdates + ((resultSizeAccumId, resultSize)) - } - /** * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the * DAG Scheduler. @@ -678,7 +652,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var accumUpdates: Map[Long, Any] = Map[Long, Any]() + var accumUpdates: Seq[AccumulableInfo] = null val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { @@ -694,7 +668,7 @@ private[spark] class TaskSetManager( case ef: ExceptionFailure => // ExceptionFailure's might have accumulator updates - accumUpdates = ef.accumulatorUpdates + accumUpdates = ef.accumUpdates if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index d3d1b67e6036..87996429334d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -237,7 +237,11 @@ private[v1] object AllStagesResource { } def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { - new AccumulableInfo(acc.id, acc.name, acc.update, acc.value) + new AccumulableInfo( + acc.id, + acc.name, + acc.update.map(_.toString), + acc.value.map(_.toString).orNull) } def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { 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 d05e55a17182..b1b80539c032 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -286,8 +286,8 @@ private[spark] object JsonProtocol { def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { ("ID" -> accumulableInfo.id) ~ ("Name" -> accumulableInfo.name) ~ - ("Update" -> accumulableInfo.update.map(new JString(_)).getOrElse(JNothing)) ~ - ("Value" -> accumulableInfo.value) ~ + ("Update" -> accumulableInfo.update.map(_.toString)) ~ + ("Value" -> accumulableInfo.value.map(_.toString)) ~ ("Internal" -> accumulableInfo.internal) } @@ -350,12 +350,13 @@ private[spark] object JsonProtocol { ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => - // TODO: serialize accumulator updates as well? val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) + val accumUpdates = JArray(exceptionFailure.accumUpdates.map(accumulableInfoToJson).toList) ("Class Name" -> exceptionFailure.className) ~ ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ - ("Full Stack Trace" -> exceptionFailure.fullStackTrace) + ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ + ("Accumulator Updates" -> accumUpdates) case taskCommitDenied: TaskCommitDenied => ("Job ID" -> taskCommitDenied.jobID) ~ ("Partition ID" -> taskCommitDenied.partitionID) ~ @@ -676,7 +677,7 @@ private[spark] object JsonProtocol { val finishTime = (json \ "Finish Time").extract[Long] val failed = (json \ "Failed").extract[Boolean] val accumulables = (json \ "Accumulables").extractOpt[Seq[JValue]] match { - case Some(values) => values.map(accumulableInfoFromJson(_)) + case Some(values) => values.map(accumulableInfoFromJson) case None => Seq[AccumulableInfo]() } @@ -692,10 +693,10 @@ private[spark] object JsonProtocol { def accumulableInfoFromJson(json: JValue): AccumulableInfo = { val id = (json \ "ID").extract[Long] val name = (json \ "Name").extract[String] - val update = Utils.jsonOption(json \ "Update").map(_.extract[String]) - val value = (json \ "Value").extract[String] + val update = (json \ "Update").extractOpt[String] + val value = (json \ "Value").extractOpt[String] val internal = (json \ "Internal").extractOpt[Boolean].getOrElse(false) - AccumulableInfo(id, name, update, value, internal) + new AccumulableInfo(id, name, update, value, internal) } def taskMetricsFromJson(json: JValue): TaskMetrics = { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 8aa38cda29ec..a0c86d76bcaf 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -162,13 +162,11 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex test("internal accumulators in TaskContext") { sc = new SparkContext("local", "test") val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null) - val accumulators = taskContext.taskMetrics.accumulators - val accumulatorValues = taskContext.taskMetrics.accumulatorUpdates() - assert(accumulators.size > 0) - assert(accumulators.forall(_.isInternal)) + val accumUpdates = taskContext.taskMetrics.accumulatorUpdates() + assert(accumUpdates.size > 0) + assert(accumUpdates.forall(_.internal)) val testAccum = taskContext.taskMetrics.getAccum(TEST_ACCUM) - assert(accumulatorValues.size === accumulators.size) - assert(accumulatorValues.contains(testAccum.id)) + assert(accumUpdates.exists(_.id == testAccum.id)) } test("internal accumulators in a stage") { @@ -189,13 +187,13 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(taskInfos.size === numPartitions) // The accumulator values should be merged in the stage val stageAccum = findTestAccum(stageInfos.head.accumulables.values) - assert(stageAccum.value.toLong === numPartitions) + assert(stageAccum.value.get.toString.toLong === numPartitions) // The accumulator should be updated locally on each task val taskAccumValues = taskInfos.map { taskInfo => val taskAccum = findTestAccum(taskInfo.accumulables) assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong + assert(taskAccum.update.get.toString.toLong === 1L) + taskAccum.value.get.toString.toLong } // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) @@ -235,9 +233,9 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex (findTestAccum(stageInfos(0).accumulables.values), findTestAccum(stageInfos(1).accumulables.values), findTestAccum(stageInfos(2).accumulables.values)) - assert(firstStageAccum.value.toLong === numPartitions) - assert(secondStageAccum.value.toLong === numPartitions * 10) - assert(thirdStageAccum.value.toLong === numPartitions * 2 * 100) + assert(firstStageAccum.value.get.toString.toLong === numPartitions) + assert(secondStageAccum.value.get.toString.toLong === numPartitions * 10) + assert(thirdStageAccum.value.get.toString.toLong === numPartitions * 2 * 100) } rdd.count() } @@ -286,14 +284,14 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(taskInfos.size === numPartitions + numFailedPartitions) val stageAccum = findTestAccum(stageInfos.head.accumulables.values) // We should not double count values in the merged accumulator - assert(stageAccum.value.toLong === numPartitions) + assert(stageAccum.value.get.toString.toLong === numPartitions) val taskAccumValues = taskInfos.flatMap { taskInfo => if (!taskInfo.failed) { // If a task succeeded, its update value should always be 1 val taskAccum = findTestAccum(taskInfo.accumulables) assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - Some(taskAccum.value.toLong) + assert(taskAccum.update.get.toString.toLong === 1L) + Some(taskAccum.value.get.toString.toLong) } else { // If a task failed, we should not get its accumulator values assert(taskInfo.accumulables.isEmpty) @@ -322,7 +320,9 @@ private[spark] object AccumulatorSuite { sc.addSparkListener(listener) testBody val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) - val isSet = accums.exists { a => a.name == PEAK_EXECUTION_MEMORY && a.value.toLong > 0 } + val isSet = accums.exists { a => + a.name == PEAK_EXECUTION_MEMORY && a.value.get.toString.toLong > 0 + } if (!isSet) { throw new TestFailedException(s"peak execution memory accumulator not set in '$testName'", 0) } 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 928a98a127c4..5e69fedd644b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -265,7 +265,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(makeCompletionEvent( - taskSet.tasks(i), result._1, result._2, Map[Long, Any](accumId -> 1))) + taskSet.tasks(i), + result._1, + result._2, + Seq(new AccumulableInfo(accumId, "", Some(1), None, internal = false)))) } } } @@ -342,9 +345,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } test("equals and hashCode AccumulableInfo") { - val accInfo1 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, true) - val accInfo2 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, false) - val accInfo3 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, false) + val accInfo1 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), true) + val accInfo2 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), false) + val accInfo3 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), false) assert(accInfo1 !== accInfo2) assert(accInfo2 === accInfo3) assert(accInfo2.hashCode() === accInfo3.hashCode()) @@ -1551,7 +1554,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou test("accumulator not calculated for resubmitted result stage") { // just for register - val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) + val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam, Some("namanama"), internal = false) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) @@ -1920,10 +1923,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou task: Task[_], reason: TaskEndReason, result: Any, - extraAccumUpdates: Map[Long, Any] = Map[Long, Any](), + extraAccumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { - val accumUpdates = - task.initialAccumulators.map { a => (a.id, a.zero) }.toMap ++ extraAccumUpdates + val accumUpdates = task.initialAccumulators.map { + a => new AccumulableInfo(a.id, a.name.get, Some(a.zero), None, a.isInternal) + } ++ extraAccumUpdates CompletionEvent(task, reason, result, accumUpdates, taskInfo) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 24f275e916bd..484632c0ea5c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -37,7 +37,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], + accumUpdates: Seq[AccumulableInfo], taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -165,7 +165,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - val accumUpdates = taskSet.tasks.head.initialAccumulators.map { a => (a.id, 0L) }.toMap + val accumUpdates = taskSet.tasks.head.initialAccumulators.map { a => + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal) + } // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have @@ -173,7 +175,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(taskOption.isDefined) // Tell it the task has finished - manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdates), 1L) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdates)) assert(sched.endedTasks(0) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -183,8 +185,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val accumUpdatesByTask: Array[Map[Long, Long]] = - taskSet.tasks.map { _.initialAccumulators.map { a => (a.id, 0L) }.toMap } + val accumUpdatesByTask: Array[Seq[AccumulableInfo]] = + taskSet.tasks.map { _.initialAccumulators.map { a => + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal) + }} // First three offers should all find tasks for (i <- 0 until 3) { @@ -199,14 +203,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) // Finish the first two tasks - manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0)), 1L) - manager.handleSuccessfulTask(1, createTaskResult(1, accumUpdatesByTask(1)), 1L) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) + manager.handleSuccessfulTask(1, createTaskResult(1, accumUpdatesByTask(1))) assert(sched.endedTasks(0) === Success) assert(sched.endedTasks(1) === Success) assert(!sched.finishedManagers.contains(manager)) // Finish the last task - manager.handleSuccessfulTask(2, createTaskResult(2, accumUpdatesByTask(2)), 1L) + manager.handleSuccessfulTask(2, createTaskResult(2, accumUpdatesByTask(2))) assert(sched.endedTasks(2) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -789,7 +793,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private def createTaskResult( id: Int, - accumUpdates: Map[Long, Any] = Map[Long, Any]()): DirectTaskResult[Int] = { + accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo]): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } 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 86699e7f5695..1e025bdf3916 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -81,7 +81,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) val peakExecutionMemory = 10 taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY, - Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true) + Some(peakExecutionMemory.toString), Some((peakExecutionMemory * taskId).toString), true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) taskInfo.markSuccessful() diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0debcd8cea35..e71d421cbcd5 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -142,8 +142,7 @@ class JsonProtocolSuite extends SparkFunSuite { "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskEndReason - val exceptionFailure = new ExceptionFailure( - exception, scala.collection.immutable.Map[Long, Any]()) + val exceptionFailure = new ExceptionFailure(exception, Seq.empty[AccumulableInfo]) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) @@ -744,7 +743,7 @@ class JsonProtocolSuite extends SparkFunSuite { } private def makeAccumulableInfo(id: Int, internal: Boolean = false): AccumulableInfo = - AccumulableInfo(id, " Accumulable " + id, Some("delta" + id), "val" + id, internal) + new AccumulableInfo(id, " Accumulable " + id, Some("delta" + id), Some("val" + id), internal) /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index cd5613692708..f0ff286ab7f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -177,7 +177,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi taskId: Long, stageId: Int, stageAttemptID: Int, - accumulatorUpdates: Map[Long, Any], + accumulatorUpdates: Seq[AccumulableInfo], finishTask: Boolean): Unit = { _stageIdToStageMetrics.get(stageId) match { @@ -289,8 +289,10 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi for (stageId <- executionUIData.stages; stageMetrics <- _stageIdToStageMetrics.get(stageId).toIterable; taskMetrics <- stageMetrics.taskIdToMetricUpdates.values; - accumulatorUpdate <- taskMetrics.accumulatorUpdates.toSeq) yield { - accumulatorUpdate + accumulatorUpdate <- taskMetrics.accumulatorUpdates) yield { + assert(accumulatorUpdate.update.isDefined, s"accumulator update from " + + s"task did not have a partial value: ${accumulatorUpdate.name}") + (accumulatorUpdate.id, accumulatorUpdate.update.get) } }.filter { case (id, _) => executionUIData.accumulatorMetrics.contains(id) } mergeAccumulatorUpdates(accumulatorUpdates, accumulatorId => @@ -329,8 +331,9 @@ private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) taskEnd.stageId, taskEnd.stageAttemptId, taskEnd.taskInfo.accumulables.map { acc => - (acc.id, new LongSQLMetricValue(acc.update.getOrElse("0").toLong)) - }.toMap, + val newValue = new LongSQLMetricValue(acc.update.map(_.toString).getOrElse("0").toLong) + acc.copy(update = Some(newValue)) + }, finishTask = true) } @@ -406,4 +409,4 @@ private[ui] class SQLStageMetrics( private[ui] class SQLTaskMetrics( val attemptId: Long, // TODO not used yet var finished: Boolean, - var accumulatorUpdates: Map[Long, Any]) + var accumulatorUpdates: Seq[AccumulableInfo]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index ca50279b40b9..97f1273624ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -71,8 +71,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { private def createTaskMetrics(accumulatorUpdates: Map[Long, Long]): TaskMetrics = { val metrics = mock(classOf[TaskMetrics]) - when(metrics.accumulatorUpdates()) - .thenReturn(accumulatorUpdates.mapValues(new LongSQLMetricValue(_))) + when(metrics.accumulatorUpdates()).thenReturn(accumulatorUpdates.map { case (id, update) => + new AccumulableInfo(id, "", Some(new LongSQLMetricValue(update)), None, internal = true) + }.toSeq) metrics } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index b46b0d2f6040..e60739435ddb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -140,7 +140,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { .filter(_._2.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) assert(peakMemoryAccumulator.size == 1) - peakMemoryAccumulator.head._2.value.toLong + peakMemoryAccumulator.head._2.value.get.toString.toLong } assert(sparkListener.getCompletedStageInfos.length == 2) From fbaf32fdba67a69ca81775ee99e8f3abcd664b72 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 11 Jan 2016 15:03:51 -0800 Subject: [PATCH 33/65] Always send Seq[AccumulableInfo] instead of TaskMetrics Now heartbeats go through the former. This is important because accumulators use how many times they are serialized to determine whether they're on the driver or the executor. A lot of code cleanup opportunities here... --- .../scala/org/apache/spark/Accumulators.scala | 24 ++----- .../org/apache/spark/HeartbeatReceiver.scala | 6 +- .../scala/org/apache/spark/SparkEnv.scala | 3 - .../org/apache/spark/executor/Executor.scala | 19 ++--- .../apache/spark/executor/TaskMetrics.scala | 42 ++++++++++- .../apache/spark/scheduler/DAGScheduler.scala | 71 ++++--------------- .../spark/scheduler/SparkListener.scala | 4 +- .../spark/scheduler/TaskScheduler.scala | 6 +- .../spark/scheduler/TaskSchedulerImpl.scala | 12 ++-- .../spark/ui/jobs/JobProgressListener.scala | 26 ++++--- .../org/apache/spark/util/JsonProtocol.scala | 20 +++--- .../apache/spark/HeartbeatReceiverSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 8 ++- .../ui/jobs/JobProgressListenerSuite.scala | 10 +-- .../apache/spark/util/JsonProtocolSuite.scala | 21 +++--- .../sql/execution/metric/SQLMetrics.scala | 4 +- .../spark/sql/execution/ui/SQLListener.scala | 5 +- .../sql/execution/ui/SQLListenerSuite.scala | 16 ++--- 18 files changed, 144 insertions(+), 159 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6814b8a3d8a2..1b0021e8b157 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -77,10 +77,10 @@ class Accumulable[R, T] private[spark] ( val zero = param.zero(initialValue) // Zero value to be passed to executors private var deserialized = false - // In certain places we create accumulators on the executors. If we register them here then - // we will never clean them up because there's no context cleaner on the executors. E.g. we - // manually create ShuffleWriteMetrics, which is a collection of accumulators, in some places. - if (isDriver) { + // In many places we create internal accumulators without registering them with the active + // context cleaner, so these accumulators are not automatically cleaned up. To avoid leaking + // map entries, we explicitly register these internal accumulators elsewhere. + if (!internal) { Accumulators.register(this) } @@ -166,14 +166,6 @@ class Accumulable[R, T] private[spark] ( */ private[spark] def setValue(newValue: R): Unit = { value_ = newValue } - /** - * Whether we are on the driver or on the executors. - * Note: in local mode, this will inevitably return true even when we're on the executors. - */ - private def isDriver: Boolean = { - Option(SparkEnv.get).map(_.isDriver).getOrElse(true) - } - // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() @@ -417,7 +409,7 @@ private[spark] object InternalAccumulator { import AccumulatorParam._ // Prefixes used in names of internal task level metrics - private val METRICS_PREFIX = "metrics." + val METRICS_PREFIX = "internal.metrics." val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." val OUTPUT_METRICS_PREFIX = METRICS_PREFIX + "output." @@ -468,7 +460,6 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking internal metrics. - * Note: this method does not register accumulators for cleanup. */ def create(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( @@ -493,7 +484,6 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking shuffle read metrics. - * Note: this method does not register accumulators for cleanup. */ def createShuffleReadAccums(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( @@ -507,7 +497,6 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking shuffle write metrics. - * Note: this method does not register accumulators for cleanup. */ def createShuffleWriteAccums(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( @@ -518,7 +507,6 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking input metrics. - * Note: this method does not register accumulators for cleanup. */ def createInputAccums(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( @@ -529,7 +517,6 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking output metrics. - * Note: this method does not register accumulators for cleanup. */ private def createOutputAccums(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( @@ -548,6 +535,7 @@ private[spark] object InternalAccumulator { def create(sc: SparkContext): Seq[Accumulator[_]] = { val accums = create() accums.foreach { accum => + Accumulators.register(accum) sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) } accums diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index e03977828b86..c880d047d006 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} */ private[spark] case class Heartbeat( executorId: String, - taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + accumUpdates: Array[(Long, Seq[AccumulableInfo])], blockManagerId: BlockManagerId) /** @@ -119,14 +119,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) context.reply(true) // Messages received from executors - case heartbeat @ Heartbeat(executorId, taskMetrics, blockManagerId) => + case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId) => if (scheduler != null) { if (executorLastSeen.contains(executorId)) { executorLastSeen(executorId) = clock.getTimeMillis() eventLoopThread.submit(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, taskMetrics, blockManagerId) + executorId, accumUpdates, blockManagerId) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) context.reply(response) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index bdeaee5f8b0a..ec43be0e2f3a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -71,9 +71,6 @@ class SparkEnv ( val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { - // Note: always returns true in local mode - private[spark] def isDriver: Boolean = blockManager.blockManagerId.isDriver - // TODO Remove actorSystem @deprecated("Actor system is no longer supported as of 1.4.0", "1.4.0") val actorSystem: ActorSystem = _actorSystem diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4ea2198ef149..814ad4d35f32 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -424,8 +424,8 @@ private[spark] class Executor( /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { - // list of (task id, metrics) to send back to the driver - val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + // list of (task id, accumUpdates) to send back to the driver + val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulableInfo])]() val curGCTime = computeTotalGcTime() for (taskRunner <- runningTasks.values().asScala) { @@ -433,23 +433,12 @@ private[spark] class Executor( taskRunner.task.metrics.foreach { metrics => metrics.mergeShuffleReadMetrics() metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - - if (isLocal) { - // JobProgressListener will hold an reference of it during - // onExecutorMetricsUpdate(), then JobProgressListener can not see - // the changes of metrics any more, so make a deep copy of it - val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) - tasksMetrics += ((taskRunner.taskId, copiedMetrics)) - } else { - // It will be copied by serialization - tasksMetrics += ((taskRunner.taskId, metrics)) - } + accumUpdates += ((taskRunner.taskId, metrics.accumulatorUpdates())) } } } - // TODO: don't send TaskMetrics here; send accumulator updates. - val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) try { val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 9fb760ff4315..a34c30ead87a 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Accumulable, Accumulator, InternalAccumulator, SparkException} +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} @@ -338,7 +338,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser } -private[spark] object TaskMetrics { +private[spark] object TaskMetrics extends Logging { def empty: TaskMetrics = new TaskMetrics @@ -359,4 +359,42 @@ private[spark] object TaskMetrics { } } + /** + * Construct a [[TaskMetrics]] object from a list of accumulator updates. + * + * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, + * we need the latter to post task end events to listeners, so we need to reconstruct the + * metrics here on the driver. + * + * Note: If the task failed, we may return null after attempting to reconstruct the + * [[TaskMetrics]] in vain. + */ + def fromAccumulatorUpdates(taskId: Long, accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { + if (accumUpdates == null) { + return null + } + // Note: it is important that we copy the accumulators here since they are used across + // many tasks and we want to maintain a snapshot of their local task values when we post + // them to listeners downstream. Otherwise, when a new task comes in the listener may get + // a different value for an old task. + val accums = accumUpdates + .filter { info => info.update.isDefined } + .flatMap { info => + Accumulators.get(info.id).map(_.copy(info.update.get)).orElse { + logWarning(s"encountered unregistered accumulator ${info.id} " + + s"when reconstructing metrics for task $taskId.") + None + } + } + // Initial accumulators are passed into the TaskMetrics constructor first because these + // are required to be uniquely named. The rest of the accumulators from this task are + // registered later because they need not satisfy this requirement. + val (initialAccums, otherAccums) = accums.partition { a => + a.name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) + } + val metrics = new TaskMetrics(initialAccums.map(_.asInstanceOf[Accumulator[_]])) + otherAccums.foreach(metrics.registerAccumulator) + metrics + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 44d76bbccf9d..ae88c68681d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -220,9 +220,10 @@ class DAGScheduler( */ def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) + // (taskId, stageId, stageAttemptId, accumUpdates) + accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates)) blockManagerMaster.driverEndpoint.askWithRetry[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } @@ -1109,66 +1110,13 @@ class DAGScheduler( } } - /** - * Reconstruct [[TaskMetrics]] from accumulator updates. - * - * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, - * we need the latter to post task end events to listeners, so we need to reconstruct the - * metrics here on the driver. - * - * Note: If the task failed, we may return null after attempting to reconstruct the - * [[TaskMetrics]] in vain. - * - * TODO: write tests here. - */ - private def reconstructTaskMetrics( - task: Task[_], - accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { - if (accumUpdates == null) { - return null - } - val taskId = task.partitionId - try { - // Initial accumulators are passed into the TaskMetrics constructor first because these - // are required to be uniquely named. The rest of the accumulators from this task are - // registered later because they need not satisfy this requirement. - val initialAccumsMap = - task.initialAccumulators.map { a => (a.id, a) }.toMap[Long, Accumulator[_]] - val (initialAccumUpdates, otherAccumUpdates) = accumUpdates.partition { a => - assert(a.update.isDefined, "accumulator update from task should have a partial value") - initialAccumsMap.contains(a.id) - } - // Note: it is important that we copy the accumulators here since they are used across - // many tasks and we want to maintain a snapshot of their local task values when we post - // them to listeners downstream. Otherwise, when a new task comes in the listener may get - // a different value for an old task. - val newInitialAccums = initialAccumUpdates.map { a => - initialAccumsMap(a.id).copy(a.update.get) - } - val otherAccums = otherAccumUpdates.flatMap { a => - val id = a.id - Accumulators.get(id).map(_.copy(a.update.get)).orElse { - logWarning(s"Task $taskId returned unregistered accumulator $id.") - None - } - } - val metrics = new TaskMetrics(newInitialAccums.toSeq) - otherAccums.foreach(metrics.registerAccumulator) - metrics - } catch { - // Do not crash the scheduler if reconstruction fails - case NonFatal(e) => - logError(s"Error when attempting to reconstruct metrics for task $taskId", e) - null - } - } - /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task + val taskId = task.partitionId val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) @@ -1203,11 +1151,18 @@ class DAGScheduler( } // Reconstruct task metrics. Note: this may be null if the task failed. - val taskMetrics = reconstructTaskMetrics(task, event.accumUpdates) + val taskMetrics: TaskMetrics = + try { + TaskMetrics.fromAccumulatorUpdates(taskId, event.accumUpdates) + } catch { + case NonFatal(e) => + logError(s"Error when attempting to reconstruct metrics for task $taskId", e) + null + } outputCommitCoordinator.taskCompleted( stageId, - task.partitionId, + taskId, event.taskInfo.attemptNumber, // this is a task attempt number event.reason) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index f5267f58c2e4..0fe9d4f2c882 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -108,12 +108,12 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends /** * Periodic updates from executors. * @param execId executor id - * @param taskMetrics sequence of (task id, stage id, stage attempt, metrics) + * @param accumUpdates sequence of (taskId, stageId, stageAttemptId, accumUpdates) */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - taskMetrics: Seq[(Long, Int, Int, TaskMetrics)]) + accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])]) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 7c0b007db708..fccd6e069934 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -65,8 +65,10 @@ private[spark] trait TaskScheduler { * alive. Return true if the driver knows about the given block manager. Otherwise, return false, * indicating that the block manager should re-register. */ - def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean + def executorHeartbeatReceived( + execId: String, + accumUpdates: Array[(Long, Seq[AccumulableInfo])], + blockManagerId: BlockManagerId): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6e3ef0e54f0f..5baea21bad0b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -380,17 +380,17 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + accumUpdates: Array[(Long, Seq[AccumulableInfo])], blockManagerId: BlockManagerId): Boolean = { - - val metricsWithStageIds: Array[(Long, Int, Int, TaskMetrics)] = synchronized { - taskMetrics.flatMap { case (id, metrics) => + // (taskId, stageId, stageAttemptId, accumUpdates) + val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { + accumUpdates.flatMap { case (id, updates) => taskIdToTaskSetManager.get(id).map { taskSetMgr => - (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, metrics) + (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, updates) } } } - dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) + dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 71493d569cad..61dc24a37e59 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -325,12 +325,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val taskInfo = taskStart.taskInfo if (taskInfo != null) { + val metrics = new TaskMetrics val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { logWarning("Task start for unknown stage " + taskStart.stageId) new StageUIData }) stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) + stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo, Some(metrics))) } for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); @@ -381,15 +382,21 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { (None, Option(taskEnd.taskMetrics)) case e: ExceptionFailure => // Handle ExceptionFailure because we might have metrics stageData.numFailedTasks += 1 - (Some(e.toErrorString), e.metrics) + val metrics = + if (e.accumUpdates.nonEmpty) { + Some(TaskMetrics.fromAccumulatorUpdates(info.taskId, e.accumUpdates)) + } else { + None + } + (Some(e.toErrorString), metrics) case e: TaskFailedReason => // All other failure cases stageData.numFailedTasks += 1 (Some(e.toErrorString), None) } - if (!metrics.isEmpty) { + metrics.foreach { m => val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics) - updateAggregateMetrics(stageData, info.executorId, metrics.get, oldMetrics) + updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) @@ -417,8 +424,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage * aggregate metrics by calculating deltas between the currently recorded metrics and the new * metrics. - * - * TODO: no need to do this! Just use accumulators. :) */ private def updateAggregateMetrics( stageData: StageUIData, @@ -491,19 +496,18 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - for ((taskId, sid, sAttempt, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + for ((taskId, sid, sAttempt, accumUpdates) <- executorMetricsUpdate.accumUpdates) { val stageData = stageIdToData.getOrElseUpdate((sid, sAttempt), { logWarning("Metrics update for task in unknown stage " + sid) new StageUIData }) val taskData = stageData.taskData.get(taskId) + val metrics = TaskMetrics.fromAccumulatorUpdates(taskId, accumUpdates) taskData.map { t => if (!t.taskInfo.finished) { - updateAggregateMetrics(stageData, executorMetricsUpdate.execId, taskMetrics, - t.taskMetrics) - + updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.taskMetrics) // Overwrite task metrics - t.taskMetrics = Some(taskMetrics) + t.taskMetrics = Some(metrics) } } } 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 b1b80539c032..ac04d491eb1a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -233,14 +233,14 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId - val taskMetrics = metricsUpdate.taskMetrics + val accumUpdates = metricsUpdate.accumUpdates ("Event" -> Utils.getFormattedClassName(metricsUpdate)) ~ ("Executor ID" -> execId) ~ - ("Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => + ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => ("Task ID" -> taskId) ~ ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ - ("Task Metrics" -> taskMetricsToJson(metrics)) + ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) }) } @@ -621,14 +621,15 @@ private[spark] object JsonProtocol { def executorMetricsUpdateFromJson(json: JValue): SparkListenerExecutorMetricsUpdate = { val execInfo = (json \ "Executor ID").extract[String] - val taskMetrics = (json \ "Metrics Updated").extract[List[JValue]].map { json => + val accumUpdates = (json \ "Metrics Updated").extract[List[JValue]].map { json => val taskId = (json \ "Task ID").extract[Long] val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] - val metrics = taskMetricsFromJson(json \ "Task Metrics") - (taskId, stageId, stageAttemptId, metrics) + val updates = + (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) + (taskId, stageId, stageAttemptId, updates) } - SparkListenerExecutorMetricsUpdate(execInfo, taskMetrics) + SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates) } /** --------------------------------------------------------------------- * @@ -790,9 +791,10 @@ private[spark] object JsonProtocol { val stackTrace = stackTraceFromJson(json \ "Stack Trace") val fullStackTrace = Utils.jsonOption(json \ "Full Stack Trace"). map(_.extract[String]).orNull - val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) + val accumUpdates = (json \ "Accumulator Updates") + .extract[List[JValue]].map(accumulableInfoFromJson) ExceptionFailure( - className, description, stackTrace, fullStackTrace, None, metrics = metrics) + className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled case `taskCommitDenied` => diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 18e53508406d..c7b695492fa1 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -215,14 +215,16 @@ class HeartbeatReceiverSuite val metrics = new TaskMetrics val blockManagerId = BlockManagerId(executorId, "localhost", 12345) val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( - Heartbeat(executorId, Array(1L -> metrics), blockManagerId)) + Heartbeat(executorId, Array(1L -> metrics.accumulatorUpdates()), blockManagerId)) if (executorShouldReregister) { assert(response.reregisterBlockManager) } else { assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + Matchers.eq(executorId), + Matchers.eq(Array(1L -> metrics.accumulatorUpdates())), + Matchers.eq(blockManagerId)) } } 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 5e69fedd644b..b1eb079924f9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -109,8 +109,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def executorHeartbeatReceived( + execId: String, + accumUpdates: Array[(Long, Seq[AccumulableInfo])], + blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -471,7 +473,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, TaskMetrics)], + accumUpdates: Array[(Long, Seq[AccumulableInfo])], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 50453bb6d567..18a16a25bfac 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -269,7 +269,9 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val execId = "exe-1" def makeTaskMetrics(base: Int): TaskMetrics = { - val taskMetrics = new TaskMetrics() + val accums = InternalAccumulator.create() + accums.foreach(Accumulators.register) + val taskMetrics = new TaskMetrics(accums) val shuffleReadMetrics = taskMetrics.registerTempShuffleReadMetrics() val shuffleWriteMetrics = taskMetrics.registerShuffleWriteMetrics() val inputMetrics = taskMetrics.registerInputMetrics(DataReadMethod.Hadoop) @@ -300,9 +302,9 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1237L))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( - (1234L, 0, 0, makeTaskMetrics(0)), - (1235L, 0, 0, makeTaskMetrics(100)), - (1236L, 1, 0, makeTaskMetrics(200))))) + (1234L, 0, 0, makeTaskMetrics(0).accumulatorUpdates()), + (1235L, 0, 0, makeTaskMetrics(100).accumulatorUpdates()), + (1236L, 1, 0, makeTaskMetrics(200).accumulatorUpdates())))) var stage0Data = listener.stageIdToData.get((0, 0)).get var stage1Data = listener.stageIdToData.get((1, 0)).get diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e71d421cbcd5..79ed7a68abb0 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -84,7 +84,7 @@ class JsonProtocolSuite extends SparkFunSuite { val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", Seq( (1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, - hasHadoopInput = true, hasOutput = true)))) + hasHadoopInput = true, hasOutput = true).accumulatorUpdates()))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -476,14 +476,17 @@ class JsonProtocolSuite extends SparkFunSuite { assert(e1.executorId === e1.executorId) case (e1: SparkListenerExecutorMetricsUpdate, e2: SparkListenerExecutorMetricsUpdate) => assert(e1.execId === e2.execId) - assertSeqEquals[(Long, Int, Int, TaskMetrics)](e1.taskMetrics, e2.taskMetrics, (a, b) => { - val (taskId1, stageId1, stageAttemptId1, metrics1) = a - val (taskId2, stageId2, stageAttemptId2, metrics2) = b - assert(taskId1 === taskId2) - assert(stageId1 === stageId2) - assert(stageAttemptId1 === stageAttemptId2) - assertEquals(metrics1, metrics2) - }) + assertSeqEquals[(Long, Int, Int, Seq[AccumulableInfo])]( + e1.accumUpdates, + e2.accumUpdates, + (a, b) => { + val (taskId1, stageId1, stageAttemptId1, updates1) = a + val (taskId2, stageId2, stageAttemptId2, updates2) = b + assert(taskId1 === taskId2) + assert(stageId1 === stageId2) + assert(stageAttemptId1 === stageAttemptId2) + assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) + }) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 9ebd8bfd3d07..940899886fa5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.metric -import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} +import org.apache.spark.{Accumulable, AccumulableParam, Accumulators, SparkContext} import org.apache.spark.util.Utils /** @@ -131,6 +131,8 @@ private[sql] object SQLMetrics { name: String, param: LongSQLMetricParam): LongSQLMetric = { val acc = new LongSQLMetric(name, param) + // This is an internal accumulator so we need to register it explicitly. + Accumulators.register(acc) sc.cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index f0ff286ab7f7..23d9907dce73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -139,9 +139,8 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { - for ((taskId, stageId, stageAttemptID, metrics) <- executorMetricsUpdate.taskMetrics) { - updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, metrics.accumulatorUpdates(), - finishTask = false) + for ((taskId, stageId, stageAttemptID, accumUpdates) <- executorMetricsUpdate.accumUpdates) { + updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, accumUpdates, finishTask = false) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 97f1273624ac..570d00778a96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -119,16 +119,16 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, metrics) - (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 0, createTaskMetrics(accumulatorUpdates)) + (0L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()), + (1L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, metrics) - (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 0, createTaskMetrics(accumulatorUpdates.mapValues(_ * 2))) + (0L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()), + (1L, 0, 0, createTaskMetrics(accumulatorUpdates.mapValues(_ * 2)).accumulatorUpdates()) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 3)) @@ -138,8 +138,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, metrics) - (0L, 0, 1, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 1, createTaskMetrics(accumulatorUpdates)) + (0L, 0, 1, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()), + (1L, 0, 1, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) @@ -178,8 +178,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, metrics) - (0L, 1, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 1, 0, createTaskMetrics(accumulatorUpdates)) + (0L, 1, 0, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()), + (1L, 1, 0, createTaskMetrics(accumulatorUpdates).accumulatorUpdates()) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 7)) From 08b6c86994e5e96d7e5e19ad843e84c3cf5879d7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 11 Jan 2016 16:12:17 -0800 Subject: [PATCH 34/65] Fix JsonProtocolSuite + cleanup --- .../org/apache/spark/TaskEndReason.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 5 +- .../apache/spark/scheduler/DAGScheduler.scala | 64 ++--- .../spark/scheduler/TaskSetManager.scala | 5 +- .../org/apache/spark/util/JsonProtocol.scala | 7 +- .../apache/spark/util/JsonProtocolSuite.scala | 237 +++++++++++++----- 7 files changed, 220 insertions(+), 105 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 5363a93ef020..4e08b4bd5cb3 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -20,7 +20,6 @@ package org.apache.spark import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils @@ -117,9 +116,7 @@ case class ExceptionFailure( stackTrace: Array[StackTraceElement], fullStackTrace: String, exceptionWrapper: Option[ThrowableSerializationWrapper], - accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], - // always None, kept here for backward compatibility - metrics: Option[TaskMetrics] = None) + accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo]) extends TaskFailedReason { /** diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 814ad4d35f32..3f0bc38e8d72 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -317,7 +317,7 @@ private[spark] class Executor( } catch { case _: NotSerializableException => // t is not serializable so just send the stacktrace - ser.serialize(new ExceptionFailure(t, accumulatorUpdates, false)) + ser.serialize(new ExceptionFailure(t, accumulatorUpdates, preserveCause = false)) } } execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index a34c30ead87a..1a09311ca855 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -364,15 +364,12 @@ private[spark] object TaskMetrics extends Logging { * * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, * we need the latter to post task end events to listeners, so we need to reconstruct the - * metrics here on the driver. + * metrics on the driver. * * Note: If the task failed, we may return null after attempting to reconstruct the * [[TaskMetrics]] in vain. */ def fromAccumulatorUpdates(taskId: Long, accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { - if (accumUpdates == null) { - return null - } // Note: it is important that we copy the accumulators here since they are used across // many tasks and we want to maintain a snapshot of their local task values when we post // them to listeners downstream. Otherwise, when a new task comes in the listener may get diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ae88c68681d3..ea863cd38488 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1080,33 +1080,31 @@ class DAGScheduler( val task = event.task val stage = stageIdToStage(task.stageId) var failedDuringThisOne: AccumulableInfo = null - if (event.accumUpdates != null) { - try { - event.accumUpdates.foreach { ainfo => - failedDuringThisOne = ainfo - val id = ainfo.id - assert(ainfo.update.isDefined, "accumulator from task should have a partial value") - val partialValue = ainfo.update.get - // Find the previously registered accumulator and update it - val acc: Accumulable[Any, Any] = Accumulators.get(id) match { - case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] - case None => - throw new SparkException(s"attempted to access non-existent accumulator $id") - } - acc ++= partialValue - // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && partialValue != acc.zero) { - val name = acc.name.get - stage.latestInfo.accumulables(id) = - new AccumulableInfo(id, name, None, Some(acc.value), acc.isInternal) - event.taskInfo.accumulables += - new AccumulableInfo(id, name, Some(partialValue), Some(acc.value), acc.isInternal) - } + try { + event.accumUpdates.foreach { ainfo => + failedDuringThisOne = ainfo + val id = ainfo.id + assert(ainfo.update.isDefined, "accumulator from task should have a partial value") + val partialValue = ainfo.update.get + // Find the previously registered accumulator and update it + val acc: Accumulable[Any, Any] = Accumulators.get(id) match { + case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] + case None => + throw new SparkException(s"attempted to access non-existent accumulator $id") + } + acc ++= partialValue + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && partialValue != acc.zero) { + val name = acc.name.get + stage.latestInfo.accumulables(id) = + new AccumulableInfo(id, name, None, Some(acc.value), acc.isInternal) + event.taskInfo.accumulables += + new AccumulableInfo(id, name, Some(partialValue), Some(acc.value), acc.isInternal) } - } catch { - case NonFatal(e) => - logError(s"Failed to update accumulators for task ${task.partitionId}", e) } + } catch { + case NonFatal(e) => + logError(s"Failed to update accumulators for task ${task.partitionId}", e) } } @@ -1152,12 +1150,16 @@ class DAGScheduler( // Reconstruct task metrics. Note: this may be null if the task failed. val taskMetrics: TaskMetrics = - try { - TaskMetrics.fromAccumulatorUpdates(taskId, event.accumUpdates) - } catch { - case NonFatal(e) => - logError(s"Error when attempting to reconstruct metrics for task $taskId", e) - null + if (event.accumUpdates.nonEmpty) { + try { + TaskMetrics.fromAccumulatorUpdates(taskId, event.accumUpdates) + } catch { + case NonFatal(e) => + logError(s"Error when attempting to reconstruct metrics for task $taskId", e) + null + } + } else { + null } outputCommitCoordinator.taskCompleted( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 7afffd6211ff..e245dd6cbd85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -652,7 +652,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var accumUpdates: Seq[AccumulableInfo] = null + var accumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo] val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { @@ -792,7 +792,8 @@ private[spark] class TaskSetManager( addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info) + sched.dagScheduler.taskEnded( + tasks(index), Resubmitted, null, Seq.empty[AccumulableInfo], info) } } } 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 ac04d491eb1a..0fdeb323617c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -620,7 +620,7 @@ private[spark] object JsonProtocol { } def executorMetricsUpdateFromJson(json: JValue): SparkListenerExecutorMetricsUpdate = { - val execInfo = (json \ "Executor ID").extract[String] + val execId = (json \ "Executor ID").extract[String] val accumUpdates = (json \ "Metrics Updated").extract[List[JValue]].map { json => val taskId = (json \ "Task ID").extract[Long] val stageId = (json \ "Stage ID").extract[Int] @@ -629,7 +629,7 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates) + SparkListenerExecutorMetricsUpdate(execId, accumUpdates) } /** --------------------------------------------------------------------- * @@ -789,8 +789,7 @@ private[spark] object JsonProtocol { val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] val stackTrace = stackTraceFromJson(json \ "Stack Trace") - val fullStackTrace = Utils.jsonOption(json \ "Full Stack Trace"). - map(_.extract[String]).orNull + val fullStackTrace = (json \ "Full Stack Trace").extractOpt[String].orNull val accumUpdates = (json \ "Accumulator Updates") .extract[List[JValue]].map(accumulableInfoFromJson) ExceptionFailure( diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 79ed7a68abb0..84479e10d44a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -33,6 +33,8 @@ import org.apache.spark.storage._ class JsonProtocolSuite extends SparkFunSuite { + import InternalAccumulator._ + val jobSubmissionTime = 1421191042750L val jobCompletionTime = 1421191296660L @@ -601,7 +603,7 @@ class JsonProtocolSuite extends SparkFunSuite { assert(r1.description === r2.description) assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) assert(r1.fullStackTrace === r2.fullStackTrace) - assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) + assertSeqEquals[AccumulableInfo](r1.accumUpdates, r2.accumUpdates, (a, b) => a.equals(b)) case (TaskResultLost, TaskResultLost) => case (TaskKilled, TaskKilled) => case (TaskCommitDenied(jobId1, partitionId1, attemptNumber1), @@ -746,7 +748,7 @@ class JsonProtocolSuite extends SparkFunSuite { } private def makeAccumulableInfo(id: Int, internal: Boolean = false): AccumulableInfo = - new AccumulableInfo(id, " Accumulable " + id, Some("delta" + id), Some("val" + id), internal) + new AccumulableInfo(id, "Accumulable" + id, Some("delta" + id), Some("val" + id), internal) /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is @@ -1043,7 +1045,7 @@ class JsonProtocolSuite extends SparkFunSuite { | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, | "Local Bytes Read": 1100, - | "Total Records Read" : 10 + | "Total Records Read": 10 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, @@ -1278,14 +1280,14 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", | "Internal": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false @@ -1340,14 +1342,14 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", | "Internal": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false @@ -1420,14 +1422,14 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", | "Internal": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false @@ -1518,14 +1520,14 @@ class JsonProtocolSuite extends SparkFunSuite { | "Accumulables": [ | { | "ID": 2, - | "Name": " Accumulable 2", + | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", | "Internal": false | }, | { | "ID": 1, - | "Name": " Accumulable 1", + | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false @@ -1680,52 +1682,169 @@ class JsonProtocolSuite extends SparkFunSuite { """ private val executorMetricsUpdateJsonString = - s""" - |{ - | "Event": "SparkListenerExecutorMetricsUpdate", - | "Executor ID": "exec3", - | "Metrics Updated": [ - | { - | "Task ID": 1, - | "Stage ID": 2, - | "Stage Attempt ID": 3, - | "Task Metrics": { - | "Executor Deserialize Time": 300, - | "Executor Run Time": 400, - | "Result Size": 500, - | "JVM GC Time": 600, - | "Result Serialization Time": 700, - | "Memory Bytes Spilled": 800, - | "Disk Bytes Spilled": 0, - | "Input Metrics": { - | "Data Read Method": "Hadoop", - | "Bytes Read": 2100, - | "Records Read": 21 - | }, - | "Output Metrics": { - | "Data Write Method": "Hadoop", - | "Bytes Written": 1200, - | "Records Written": 12 - | }, - | "Updated Blocks": [ - | { - | "Block ID": "rdd_0_0", - | "Status": { - | "Storage Level": { - | "Use Disk": true, - | "Use Memory": true, - | "Use ExternalBlockStore": false, - | "Deserialized": false, - | "Replication": 2 - | }, - | "Memory Size": 0, - | "ExternalBlockStore Size": 0, - | "Disk Size": 0 - | } - | } - | ] - | } - | }] - |} - """.stripMargin + s""" + |{ + | "Event": "SparkListenerExecutorMetricsUpdate", + | "Executor ID": "exec3", + | "Metrics Updated": [ + | { + | "Task ID": 1, + | "Stage ID": 2, + | "Stage Attempt ID": 3, + | "Accumulator Updates": [ + | { + | "ID": 82, + | "Name": "$EXECUTOR_DESERIALIZE_TIME", + | "Update": "300", + | "Internal": true + | }, + | { + | "ID": 83, + | "Name": "$EXECUTOR_RUN_TIME", + | "Update": "400", + | "Internal": true + | }, + | { + | "ID": 84, + | "Name": "$RESULT_SIZE", + | "Update": "500", + | "Internal": true + | }, + | { + | "ID": 85, + | "Name": "$JVM_GC_TIME", + | "Update": "600", + | "Internal": true + | }, + | { + | "ID": 86, + | "Name": "$RESULT_SERIALIZATION_TIME", + | "Update": "700", + | "Internal": true + | }, + | { + | "ID": 87, + | "Name": "$MEMORY_BYTES_SPILLED", + | "Update": "800", + | "Internal": true + | }, + | { + | "ID": 88, + | "Name": "$DISK_BYTES_SPILLED", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 89, + | "Name": "$PEAK_EXECUTION_MEMORY", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 90, + | "Name": "$UPDATED_BLOCK_STATUSES", + | "Update": "Vector((rdd_0_0,BlockStatus(StorageLevel(true, true, false, false, 2),0,0,0)))", + | "Internal": true + | }, + | { + | "ID": 91, + | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 92, + | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 93, + | "Name": "${shuffleRead.REMOTE_BYTES_READ}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 94, + | "Name": "${shuffleRead.LOCAL_BYTES_READ}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 95, + | "Name": "${shuffleRead.FETCH_WAIT_TIME}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 96, + | "Name": "${shuffleRead.RECORDS_READ}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 97, + | "Name": "${shuffleWrite.BYTES_WRITTEN}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 98, + | "Name": "${shuffleWrite.RECORDS_WRITTEN}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 99, + | "Name": "${shuffleWrite.WRITE_TIME}", + | "Update": "0", + | "Internal": true + | }, + | { + | "ID": 100, + | "Name": "${input.READ_METHOD}", + | "Update": "Hadoop", + | "Internal": true + | }, + | { + | "ID": 101, + | "Name": "${input.BYTES_READ}", + | "Update": "2100", + | "Internal": true + | }, + | { + | "ID": 102, + | "Name": "${input.RECORDS_READ}", + | "Update": "21", + | "Internal": true + | }, + | { + | "ID": 103, + | "Name": "${output.WRITE_METHOD}", + | "Update": "Hadoop", + | "Internal": true + | }, + | { + | "ID": 104, + | "Name": "${output.BYTES_WRITTEN}", + | "Update": "1200", + | "Internal": true + | }, + | { + | "ID": 105, + | "Name": "${output.RECORDS_WRITTEN}", + | "Update": "12", + | "Internal": true + | }, + | { + | "ID": 106, + | "Name": "$TEST_ACCUM", + | "Update": "0", + | "Internal": true + | } + | ] + | } + | ] + |} + """.stripMargin } From c1db0087de440b300d07d7f058fa6841815de6c8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 11 Jan 2016 18:35:29 -0800 Subject: [PATCH 35/65] More cleanups --- .../scala/org/apache/spark/Accumulators.scala | 29 +++++++-- .../apache/spark/executor/InputMetrics.scala | 2 +- .../apache/spark/executor/OutputMetrics.scala | 4 +- .../spark/executor/ShuffleReadMetrics.scala | 27 ++++---- .../spark/executor/ShuffleWriteMetrics.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 64 ++++++++++--------- .../apache/spark/scheduler/DAGScheduler.scala | 30 ++++----- .../org/apache/spark/scheduler/Task.scala | 6 +- .../spark/scheduler/TaskResultGetter.scala | 5 +- .../spark/scheduler/TaskSetManager.scala | 1 - .../org/apache/spark/status/api/v1/api.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 1 + .../UnsafeFixedWidthAggregationMapSuite.scala | 3 +- 14 files changed, 101 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 1b0021e8b157..908de3fb2a5a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -37,6 +37,10 @@ import org.apache.spark.util.Utils * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are * accumulating a set. You will add items to the set, and you will union two sets together. * + * All accumulators created on the driver to be used on the executors must be registered with + * [[Accumulators]]. This is already done automatically for accumulators created by the user. + * Internal accumulators must be explicitly registered by the caller. + * * Operations are not thread-safe. * * @param initialValue initial value of accumulator @@ -77,9 +81,9 @@ class Accumulable[R, T] private[spark] ( val zero = param.zero(initialValue) // Zero value to be passed to executors private var deserialized = false - // In many places we create internal accumulators without registering them with the active - // context cleaner, so these accumulators are not automatically cleaned up. To avoid leaking - // map entries, we explicitly register these internal accumulators elsewhere. + // In many places we create internal accumulators without access to the active context cleaner, + // so if we register them here then we may never unregister these accumulators. To avoid memory + // leaks, we require the caller to explicitly register internal accumulators elsewhere. if (!internal) { Accumulators.register(this) } @@ -371,6 +375,14 @@ private[spark] object Accumulators extends Logging { lastId } + /** + * Register an accumulator created on the driver such that it can be used on the executors. + * + * All accumulators registered here can later be used as a container for accumulating partial + * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. + * Note: if an accumulator is registered here, it should also be registered with the active + * context cleaner for cleanup so as to avoid memory leaks. + */ def register(a: Accumulable[_, _]): Unit = synchronized { originals(a.id) = new WeakReference[Accumulable[_, _]](a) } @@ -487,8 +499,8 @@ private[spark] object InternalAccumulator { */ def createShuffleReadAccums(): Seq[Accumulator[_]] = { Seq[Accumulator[_]]( - newLongMetric(shuffleRead.REMOTE_BLOCKS_FETCHED), - newLongMetric(shuffleRead.LOCAL_BLOCKS_FETCHED), + newIntMetric(shuffleRead.REMOTE_BLOCKS_FETCHED), + newIntMetric(shuffleRead.LOCAL_BLOCKS_FETCHED), newLongMetric(shuffleRead.REMOTE_BYTES_READ), newLongMetric(shuffleRead.LOCAL_BYTES_READ), newLongMetric(shuffleRead.FETCH_WAIT_TIME), @@ -548,6 +560,13 @@ private[spark] object InternalAccumulator { new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) } + /** + * Create a new Long accumulator representing an internal task metric. + */ + private def newIntMetric(name: String): Accumulator[Int] = { + newMetric[Int](0, name, IntAccumulatorParam) + } + /** * Create a new Long accumulator representing an internal task metric. */ diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 6388d1c4fee7..060ffdd15246 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -35,7 +35,7 @@ object DataReadMethod extends Enumeration with Serializable { /** * :: DeveloperApi :: - * Metrics about reading input data. + * A collection of accumulators that represents metrics about reading data from external systems. */ @DeveloperApi class InputMetrics private ( diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index c2be4ed105c3..170cc0c49b1c 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -34,7 +34,7 @@ object DataWriteMethod extends Enumeration with Serializable { /** * :: DeveloperApi :: - * Metrics about writing output data. + * A collection of accumulators that represents metrics about writing data to external systems. */ @DeveloperApi class OutputMetrics private ( @@ -61,7 +61,7 @@ class OutputMetrics private ( def recordsWritten: Long = _recordsWritten.localValue /** - * The source to which this task writes its input. + * The source to which this task writes its output. */ def writeMethod: DataWriteMethod.Value = DataWriteMethod.withName(_writeMethod.localValue) diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 5f81f63aab56..dc80e7802079 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -23,12 +23,12 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Metrics pertaining to shuffle data read in a given task. + * A collection of accumulators that represent metrics about reading shuffle data. */ @DeveloperApi class ShuffleReadMetrics private ( - _remoteBlocksFetched: Accumulator[Long], - _localBlocksFetched: Accumulator[Long], + _remoteBlocksFetched: Accumulator[Int], + _localBlocksFetched: Accumulator[Int], _remoteBytesRead: Accumulator[Long], _localBytesRead: Accumulator[Long], _fetchWaitTime: Accumulator[Long], @@ -37,8 +37,8 @@ class ShuffleReadMetrics private ( private[executor] def this(accumMap: Map[String, Accumulator[_]]) { this( - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), + TaskMetrics.getAccum[Int](accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), + TaskMetrics.getAccum[Int](accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), @@ -52,7 +52,8 @@ class ShuffleReadMetrics private ( * many places only to merge their values together later. In the future, we should revisit * whether this is needed. * - * A better alternative is [[TaskMetrics.registerTempShuffleReadMetrics]]. + * A better alternative is [[TaskMetrics.registerTempShuffleReadMetrics]] followed by + * [[TaskMetrics.mergeShuffleReadMetrics]]. */ private[spark] def this() { this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) @@ -61,12 +62,12 @@ class ShuffleReadMetrics private ( /** * Number of remote blocks fetched in this shuffle by this task. */ - def remoteBlocksFetched: Long = _remoteBlocksFetched.localValue + def remoteBlocksFetched: Int = _remoteBlocksFetched.localValue /** * Number of local blocks fetched in this shuffle by this task. */ - def localBlocksFetched: Long = _localBlocksFetched.localValue + def localBlocksFetched: Int = _localBlocksFetched.localValue /** * Total number of remote bytes read from the shuffle by this task. @@ -98,17 +99,17 @@ class ShuffleReadMetrics private ( /** * Number of blocks fetched in this shuffle by this task (remote or local). */ - def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched + def totalBlocksFetched: Int = remoteBlocksFetched + localBlocksFetched - private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) - private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) + private[spark] def incRemoteBlocksFetched(v: Int): Unit = _remoteBlocksFetched.add(v) + private[spark] def incLocalBlocksFetched(v: Int): Unit = _localBlocksFetched.add(v) private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) - private[spark] def setRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.setValue(v) - private[spark] def setLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.setValue(v) + private[spark] def setRemoteBlocksFetched(v: Int): Unit = _remoteBlocksFetched.setValue(v) + private[spark] def setLocalBlocksFetched(v: Int): Unit = _localBlocksFetched.setValue(v) private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index c012836882aa..a54961271ab8 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Metrics pertaining to shuffle data written in a given task. + * A collection of accumulators that represent metrics about writing shuffle data. */ @DeveloperApi class ShuffleWriteMetrics private ( diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 1a09311ca855..c9cfb9b5f98a 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,7 @@ package org.apache.spark.executor +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark._ @@ -36,8 +37,9 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * to the driver when the task completes. These values are then merged into the corresponding * accumulator previously registered on the driver. * - * These accumulator updates are also sent to the driver periodically (on executor heartbeat) - * or when the task failed with an exception. + * The accumulator updates are also sent to the driver periodically (on executor heartbeat) + * and when the task failed with an exception. The [[TaskMetrics]] object itself should never + * be sent to the driver directly. * * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. * Each accumulator in this initial set must be named and marked as internal. @@ -63,18 +65,19 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser * A map for quickly accessing the initial set of accumulators by name. */ private val initialAccumsMap: Map[String, Accumulator[_]] = { - initialAccums.map { a => - assert(a.name.isDefined, "initial accumulators passed to TaskMetrics should be named") + val map = new mutable.HashMap[String, Accumulator[_]] + initialAccums.foreach { a => + assert(a.name.isDefined, "initial accumulators passed to TaskMetrics must be named") val name = a.name.get assert(a.isInternal, - s"initial accumulator '$name' passed to TaskMetrics should be marked as internal") - (name, a) - }.toMap + s"initial accumulator '$name' passed to TaskMetrics must be marked as internal") + assert(!map.contains(name), + s"detected duplicate accumulator name '$name' when constructing TaskMetrics") + map(name) = a + } + map.toMap } - assert(initialAccumsMap.size == initialAccums.size, s"detected duplicate names in initial " + - s"accumulators passed to TaskMetrics:\n ${initialAccums.map(_.name.get).mkString("\n")}") - // Each metric is internally represented as an accumulator private val _executorDeserializeTime = getAccum(EXECUTOR_DESERIALIZE_TIME) private val _executorRunTime = getAccum(EXECUTOR_RUN_TIME) @@ -258,8 +261,9 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser /** * Create a temporary [[ShuffleReadMetrics]] for a particular shuffle dependency. * - * All usages are expected to be followed by a call to [[mergeShuffleReadMetrics]], - * which merges the temporary values synchronously. + * All usages are expected to be followed by a call to [[mergeShuffleReadMetrics]], which + * merges the temporary values synchronously. Otherwise, all temporary data collected will + * be lost. */ private[spark] def registerTempShuffleReadMetrics(): ShuffleReadMetrics = synchronized { val readMetrics = new ShuffleReadMetrics @@ -312,27 +316,25 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser new AccumulableInfo(a.id, a.name.orNull, Some(a.localValue), None, a.isInternal) } - /** - * Return whether some accumulators with the given prefix have already been set. - * This only considers the initial set of accumulators passed into the constructor. - */ - private def accumsAlreadySet(prefix: String): Boolean = { - initialAccumsMap.filterKeys(_.startsWith(prefix)).values.exists { a => a.localValue != a.zero } - } - // If we are reconstructing this TaskMetrics on the driver, some metrics may already be set. // If so, initialize all relevant metrics classes so listeners can access them downstream. - if (accumsAlreadySet(SHUFFLE_READ_METRICS_PREFIX)) { - _shuffleReadMetrics = Some(new ShuffleReadMetrics(initialAccumsMap)) - } - if (accumsAlreadySet(SHUFFLE_WRITE_METRICS_PREFIX)) { - _shuffleWriteMetrics = Some(new ShuffleWriteMetrics(initialAccumsMap)) - } - if (accumsAlreadySet(OUTPUT_METRICS_PREFIX)) { - _outputMetrics = Some(new OutputMetrics(initialAccumsMap)) - } - if (accumsAlreadySet(INPUT_METRICS_PREFIX)) { - _inputMetrics = Some(new InputMetrics(initialAccumsMap)) + { + var (hasShuffleRead, hasShuffleWrite, hasInput, hasOutput) = (false, false, false, false) + initialAccums + .filter { a => a.localValue != a.zero } + .map { a => + a.name.get match { + case sr if sr.startsWith(SHUFFLE_READ_METRICS_PREFIX) => hasShuffleRead = true + case sw if sw.startsWith(SHUFFLE_WRITE_METRICS_PREFIX) => hasShuffleWrite = true + case in if in.startsWith(INPUT_METRICS_PREFIX) => hasInput = true + case out if out.startsWith(OUTPUT_METRICS_PREFIX) => hasOutput = true + case _ => + } + } + if (hasShuffleRead) { _shuffleReadMetrics = Some(new ShuffleReadMetrics(initialAccumsMap)) } + if (hasShuffleWrite) { _shuffleWriteMetrics = Some(new ShuffleWriteMetrics(initialAccumsMap)) } + if (hasInput) { _inputMetrics = Some(new InputMetrics(initialAccumsMap)) } + if (hasOutput) { _outputMetrics = Some(new OutputMetrics(initialAccumsMap)) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ea863cd38488..055786452487 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1073,20 +1073,18 @@ class DAGScheduler( } /** - * Merge local values from a task into the corresponding accumulator previously - * registered here on the driver. + * Merge local values from a task into the corresponding accumulators previously registered + * here on the driver. */ private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task val stage = stageIdToStage(task.stageId) - var failedDuringThisOne: AccumulableInfo = null try { event.accumUpdates.foreach { ainfo => - failedDuringThisOne = ainfo - val id = ainfo.id assert(ainfo.update.isDefined, "accumulator from task should have a partial value") + val id = ainfo.id val partialValue = ainfo.update.get - // Find the previously registered accumulator and update it + // Find the corresponding accumulator on the driver and update it val acc: Accumulable[Any, Any] = Accumulators.get(id) match { case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] case None => @@ -1119,25 +1117,23 @@ class DAGScheduler( val taskType = Utils.getFormattedClassName(task) // Note: the following events must occur in this order: - // (1) Update accumulator values based on updates from this task + // (1) Merge accumulator updates from this task // (2) Reconstruct TaskMetrics // (3) Post SparkListenerTaskEnd event // (4) Post SparkListenerStageCompleted / SparkListenerJobEnd event - // Update accumulator values based on updates from this task. Note: we must do this before - // reconstructing TaskMetrics, otherwise the TaskMetrics will not have the updated metrics. - // This is needed for the SQL UI, for instance. + // Merge accumulator updates from this task. We must do this before reconstructing + // TaskMetrics or else the listeners downstream will not receive the most recent values. if (stageIdToStage.contains(stageId)) { val stage = stageIdToStage(stageId) - // We should should update registered accumulators if this task succeeded or failed with - // an exception. In the latter case executors may still send back some accumulators, - // so we should try our best to collect the values. TODO: write a test. + // We should do this if the task either succeeded or failed with an exception. In the + // latter case executors may still send back accumulator updates, so we should try our + // best to collect the values. TODO: write a test. val shouldUpdateAccums = event.reason match { case Success => task match { - case rt: ResultTask[_, _] => - // This being true means the job has not finished yet - stage.asInstanceOf[ResultStage].activeJob.isDefined + // This being true means the job has not finished yet + case rt: ResultTask[_, _] => stage.asInstanceOf[ResultStage].activeJob.isDefined case smt: ShuffleMapTask => true } case _: ExceptionFailure => true @@ -1148,7 +1144,7 @@ class DAGScheduler( } } - // Reconstruct task metrics. Note: this may be null if the task failed. + // Reconstruct task metrics. Note: this may be null if the task has failed. val taskMetrics: TaskMetrics = if (event.accumUpdates.nonEmpty) { try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 5d74b0cfaf23..8b6cf92bc90b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -57,9 +57,9 @@ private[spark] abstract class Task[T]( * @return the result of the task along with updates of Accumulators. */ final def run( - taskAttemptId: Long, - attemptNumber: Int, - metricsSystem: MetricsSystem): T = { + taskAttemptId: Long, + attemptNumber: Int, + metricsSystem: MetricsSystem): T = { context = new TaskContextImpl( stageId, partitionId, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 0509b1004903..e39eccf7f120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -85,8 +85,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } // Set the task result size in the accumulator updates received from the executors. - // If we did this on the executors we would have to serialize the result again after - // updating the size, which is potentially expensive. TODO: write a test. + // We need to do this here on the driver because if we did this on the executors then + // we would have to serialize the result again after updating the size. + // TODO: write a test. result.accumUpdates = result.accumUpdates.map { ainfo => if (ainfo.name == InternalAccumulator.RESULT_SIZE) { assert(ainfo.update.getOrElse(0L) == 0L, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e245dd6cbd85..a74c7f069374 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -614,7 +614,6 @@ private[spark] class TaskSetManager( val index = info.index info.markSuccessful() removeRunningTask(tid) - // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not // "deserialize" the value when holding a lock to avoid blocking other threads. So we call diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 28b5b173f02e..5feb1dc2e5b7 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -178,11 +178,11 @@ class OutputMetrics private[spark]( val recordsWritten: Long) class ShuffleReadMetrics private[spark]( - val remoteBlocksFetched: Long, - val localBlocksFetched: Long, + val remoteBlocksFetched: Int, + val localBlocksFetched: Int, val fetchWaitTime: Long, val remoteBytesRead: Long, - val totalBlocksFetched: Long, + val totalBlocksFetched: Int, val recordsRead: Long) class ShuffleWriteMetrics private[spark]( 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 0fdeb323617c..093c42b4b439 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -716,8 +716,8 @@ private[spark] object JsonProtocol { // Shuffle read metrics Utils.jsonOption(json \ "Shuffle Read Metrics").foreach { readJson => val readMetrics = metrics.registerTempShuffleReadMetrics() - readMetrics.setRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Long]) - readMetrics.setLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Long]) + readMetrics.setRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Int]) + readMetrics.setLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Int]) readMetrics.setRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) readMetrics.setLocalBytesRead((readJson \ "Local Bytes Read").extractOpt[Long].getOrElse(0L)) readMetrics.setFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) 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 9ecbffbf715c..077b87195aaf 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1630,6 +1630,7 @@ private[spark] object Utils extends Logging { /** Return an option that translates JNothing to None */ def jsonOption(json: JValue): Option[JValue] = { + // TODO: this is useless; we can just use extractOpt, which looks cleaner. json match { case JNothing => None case value: JValue => Some(value) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index e90ed3de8d0e..c7df8b51e2f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -23,14 +23,13 @@ import scala.util.control.NonFatal import org.scalatest.Matchers -import org.apache.spark._ +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.sql.types.StructField /** * Test suite for [[UnsafeFixedWidthAggregationMap]]. From 4d681fa04f2c4d8ea5d22b03a15724e7394cbc0e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 11 Jan 2016 23:43:08 -0800 Subject: [PATCH 36/65] Fix style --- core/src/main/scala/org/apache/spark/Accumulators.scala | 6 +++++- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 2 ++ 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 908de3fb2a5a..98009a626a33 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -25,7 +25,7 @@ import scala.ref.WeakReference import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.storage.{BlockStatus, BlockId} +import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils /** @@ -439,6 +439,8 @@ private[spark] object InternalAccumulator { val UPDATED_BLOCK_STATUSES = METRICS_PREFIX + "updatedBlockStatuses" val TEST_ACCUM = METRICS_PREFIX + "testAccumulator" + // scalastyle:off + // Names of shuffle read metrics object shuffleRead { val REMOTE_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "remoteBlocksFetched" @@ -470,6 +472,8 @@ private[spark] object InternalAccumulator { val RECORDS_READ = INPUT_METRICS_PREFIX + "recordsRead" } + // scalastyle:on + /** * Accumulators for tracking internal metrics. */ 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 b1eb079924f9..ea9e0c95681e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1556,7 +1556,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou test("accumulator not calculated for resubmitted result stage") { // just for register - val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam, Some("namanama"), internal = false) + val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 84479e10d44a..c25302f30608 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1681,6 +1681,7 @@ class JsonProtocolSuite extends SparkFunSuite { |} """ + // scalastyle:off private val executorMetricsUpdateJsonString = s""" |{ @@ -1847,4 +1848,5 @@ class JsonProtocolSuite extends SparkFunSuite { | ] |} """.stripMargin + // scalastyle:on } From b00318b3a109bdbbd5bde394325d35854fb5ee88 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 10:56:17 -0800 Subject: [PATCH 37/65] Fix MiMa --- project/MimaExcludes.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 0d5f938d9ef5..3c7284a0889f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -135,6 +135,15 @@ object MimaExcludes { ) ++ Seq( // SPARK-12510 Refactor ActorReceiver to support Java ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-10620 Migrate TaskMetrics to accumulators + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.internalMetricsToAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.collectInternalAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.collectAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulable.this") ) case v if v.startsWith("1.6") => Seq( From e060ae2973dfb013f272ad90450de3b34033c14a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 13:13:20 -0800 Subject: [PATCH 38/65] Fix MiMa --- project/MimaExcludes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 49b1eb4ddbb0..bdb2dfaed62d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -155,7 +155,8 @@ object MimaExcludes { "org.apache.spark.TaskContext.collectInternalAccumulators"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.TaskContext.collectAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulable.this") + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulable.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.this") ) case v if v.startsWith("1.6") => Seq( From c94420b3552fd15fe786ecc02e6f8d70c8d2f316 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 13:17:16 -0800 Subject: [PATCH 39/65] Rename updatedBlocks -> updatedBlockStatuses Just for consistency. --- .../main/scala/org/apache/spark/CacheManager.scala | 2 +- .../org/apache/spark/executor/TaskMetrics.scala | 10 ++++++---- .../org/apache/spark/memory/StorageMemoryPool.scala | 2 +- .../apache/spark/storage/StorageStatusListener.scala | 2 +- .../org/apache/spark/ui/storage/StorageTab.scala | 4 ++-- .../scala/org/apache/spark/util/JsonProtocol.scala | 4 ++-- .../scala/org/apache/spark/CacheManagerSuite.scala | 2 +- .../spark/storage/StorageStatusListenerSuite.scala | 12 ++++++------ .../apache/spark/ui/storage/StorageTabSuite.scala | 8 ++++---- .../org/apache/spark/util/JsonProtocolSuite.scala | 4 ++-- 10 files changed, 26 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 2139ce3b7e06..f527f22ef634 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -76,7 +76,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // Otherwise, cache the values and keep track of any updates in block statuses val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) - context.taskMetrics().incUpdatedBlocks(updatedBlocks) + context.taskMetrics().incUpdatedBlockStatuses(updatedBlocks) new InterruptibleIterator(context, cachedValues) } finally { loading.synchronized { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index c9cfb9b5f98a..3d1bd826bcc9 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -87,7 +87,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser private val _memoryBytesSpilled = getAccum(MEMORY_BYTES_SPILLED) private val _diskBytesSpilled = getAccum(DISK_BYTES_SPILLED) private val _peakExecutionMemory = getAccum(PEAK_EXECUTION_MEMORY) - private val _updatedBlocks = + private val _updatedBlockStatuses = TaskMetrics.getAccum[Seq[(BlockId, BlockStatus)]](initialAccumsMap, UPDATED_BLOCK_STATUSES) /** @@ -136,7 +136,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlocks: Seq[(BlockId, BlockStatus)] = _updatedBlocks.localValue + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue private[spark] def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) private[spark] def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) @@ -146,8 +146,10 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser private[spark] def incMemoryBytesSpilled(v: Long) = _memoryBytesSpilled.add(v) private[spark] def incDiskBytesSpilled(v: Long) = _diskBytesSpilled.add(v) private[spark] def incPeakExecutionMemory(v: Long) = _peakExecutionMemory.add(v) - private[spark] def incUpdatedBlocks(v: Seq[(BlockId, BlockStatus)]) = _updatedBlocks.add(v) - private[spark] def setUpdatedBlocks(v: Seq[(BlockId, BlockStatus)]) = _updatedBlocks.setValue(v) + private[spark] def incUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]) = + _updatedBlockStatuses.add(v) + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]) = + _updatedBlockStatuses.setValue(v) /* ============================ * diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 98f67e5631d5..79411e7eb6f1 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -88,7 +88,7 @@ private[memory] class StorageMemoryPool(lock: Object) extends MemoryPool(lock) w if (numBytesToFree > 0) { memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, evictedBlocks) // Register evicted blocks, if any, with the active task metrics - Option(TaskContext.get()).foreach(_.taskMetrics().incUpdatedBlocks(evictedBlocks)) + Option(TaskContext.get()).foreach(_.taskMetrics().incUpdatedBlockStatuses(evictedBlocks)) } // NOTE: If the memory store evicts blocks, then those evictions will synchronously call // back into this StorageMemoryPool in order to free memory. Therefore, these variables diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index ee73b04ed630..d98aae8ff0c6 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -63,7 +63,7 @@ class StorageStatusListener extends SparkListener { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { - val updatedBlocks = metrics.updatedBlocks + val updatedBlocks = metrics.updatedBlockStatuses if (updatedBlocks.length > 0) { updateStorageStatus(info.executorId, updatedBlocks) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 126fbee67a97..f1e28b4e1e9c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -63,8 +63,8 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val metrics = taskEnd.taskMetrics - if (metrics != null && metrics.updatedBlocks.nonEmpty) { - updateRDDInfo(metrics.updatedBlocks) + if (metrics != null && metrics.updatedBlockStatuses.nonEmpty) { + updateRDDInfo(metrics.updatedBlockStatuses) } } 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 093c42b4b439..16816b777986 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -320,7 +320,7 @@ private[spark] object JsonProtocol { ("Records Written" -> om.recordsWritten) }.getOrElse(JNothing) val updatedBlocks = - JArray(taskMetrics.updatedBlocks.toList.map { case (id, status) => + JArray(taskMetrics.updatedBlockStatuses.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) }) @@ -753,7 +753,7 @@ private[spark] object JsonProtocol { // Updated blocks Utils.jsonOption(json \ "Updated Blocks").foreach { blocksJson => - metrics.setUpdatedBlocks(blocksJson.extract[List[JValue]].map { blockJson => + metrics.setUpdatedBlockStatuses(blocksJson.extract[List[JValue]].map { blockJson => val id = BlockId((blockJson \ "Block ID").extract[String]) val status = blockStatusFromJson(blockJson \ "Status") (id, status) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index bd49e1e349e6..3292d2aa6640 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -95,6 +95,6 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before cacheManager = sc.env.cacheManager val context = TaskContext.empty() cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) - assert(context.taskMetrics.updatedBlocks.size === 2) + assert(context.taskMetrics.updatedBlockStatuses.size === 2) } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 9ea9b749c976..82aadf5fed14 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -85,8 +85,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.setUpdatedBlocks(Seq(block1, block2)) - taskMetrics2.setUpdatedBlocks(Seq(block3)) + taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2)) + taskMetrics2.setUpdatedBlockStatuses(Seq(block3)) // Task end with new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) @@ -108,8 +108,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) - taskMetrics1.setUpdatedBlocks(Seq(droppedBlock1, droppedBlock3)) - taskMetrics2.setUpdatedBlocks(Seq(droppedBlock2, droppedBlock3)) + taskMetrics1.setUpdatedBlockStatuses(Seq(droppedBlock1, droppedBlock3)) + taskMetrics2.setUpdatedBlockStatuses(Seq(droppedBlock2, droppedBlock3)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) @@ -133,8 +133,8 @@ class StorageStatusListenerSuite extends SparkFunSuite { val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.setUpdatedBlocks(Seq(block1, block2)) - taskMetrics2.setUpdatedBlocks(Seq(block3)) + taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2)) + taskMetrics2.setUpdatedBlockStatuses(Seq(block3)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index bbf3aa2ed185..cee1842b313c 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -127,7 +127,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Task end with a few new persisted blocks, some from the same RDD val metrics1 = new TaskMetrics - metrics1.setUpdatedBlocks(Seq( + metrics1.setUpdatedBlockStatuses(Seq( (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L, 0L)), (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L, 0L)), (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), @@ -149,7 +149,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { // Task end with a few dropped blocks val metrics2 = new TaskMetrics - metrics2.setUpdatedBlocks(Seq( + metrics2.setUpdatedBlockStatuses(Seq( (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L, 0L)), (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L, 0L)), (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist @@ -177,8 +177,8 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { val taskMetrics1 = new TaskMetrics val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L, 0L)) val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L, 0L)) - taskMetrics0.setUpdatedBlocks(Seq(block0)) - taskMetrics1.setUpdatedBlocks(Seq(block1)) + taskMetrics0.setUpdatedBlockStatuses(Seq(block0)) + taskMetrics1.setUpdatedBlockStatuses(Seq(block1)) bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener.rddInfoList.size === 0) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c25302f30608..1f50251cb77b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -559,7 +559,7 @@ class JsonProtocolSuite extends SparkFunSuite { metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) assertOptionEquals( metrics1.inputMetrics, metrics2.inputMetrics, assertInputMetricsEquals) - assertBlocksEquals(metrics1.updatedBlocks, metrics2.updatedBlocks) + assertBlocksEquals(metrics1.updatedBlockStatuses, metrics2.updatedBlockStatuses) } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { @@ -797,7 +797,7 @@ class JsonProtocolSuite extends SparkFunSuite { sw.incRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) } // Make at most 6 blocks - t.setUpdatedBlocks((1 to (e % 5 + 1)).map { i => + t.setUpdatedBlockStatuses((1 to (e % 5 + 1)).map { i => (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i, c%i)) }.toSeq) t From 94783d0a498de2a5821c49c95706f8d9b8032448 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 13:44:52 -0800 Subject: [PATCH 40/65] Fix wrong comment by removing it :) --- .../scala/org/apache/spark/Accumulators.scala | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 98009a626a33..06350ef5dba8 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -560,29 +560,18 @@ private[spark] object InternalAccumulator { /** * Create a new accumulator representing an internal task metric. */ - private def newMetric[T](initialValue: T, name: String, param: AccumulatorParam[T]) = { + private def newMetric[T]( + initialValue: T, + name: String, + param: AccumulatorParam[T]): Accumulator[T] = { new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) } - /** - * Create a new Long accumulator representing an internal task metric. - */ - private def newIntMetric(name: String): Accumulator[Int] = { + private def newIntMetric(name: String): Accumulator[Int] = newMetric[Int](0, name, IntAccumulatorParam) - } - - /** - * Create a new Long accumulator representing an internal task metric. - */ - private def newLongMetric(name: String): Accumulator[Long] = { + private def newLongMetric(name: String): Accumulator[Long] = newMetric[Long](0L, name, LongAccumulatorParam) - } - - /** - * Create a new String accumulator representing an internal task metric. - */ - private def newStringMetric(name: String): Accumulator[String] = { + private def newStringMetric(name: String): Accumulator[String] = newMetric[String]("", name, StringAccumulatorParam) - } } From b271205c6c2b4170d5abd2ebb52f48fa8463112d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 14:01:38 -0800 Subject: [PATCH 41/65] Fix StringAccumulatorParam semantics InputMetrics#readMethod is a string. We don't want to accumulate the read method of one task with that of another task by concatenating them, otherwise the accumulated value might look something like "HadoopHadoopHadoopHadoopHadoop...", which makes no sense. For our purposes, the String param now always overwrites the old value with the new value. --- core/src/main/scala/org/apache/spark/Accumulators.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 06350ef5dba8..48f72dbc0aff 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -343,8 +343,11 @@ object AccumulatorParam { def zero(initialValue: Float): Float = 0f } - private[spark] implicit object StringAccumulatorParam extends AccumulatorParam[String] { - def addInPlace(t1: String, t2: String): String = t1 + t2 + // Note: when merging values, this param just adopts the newer value. This is used only + // internally for things that shouldn't really be accumulated across tasks, like input + // read method, which should be the same across all tasks in the same stage. + private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { + def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" } From 8065b250fffed9566fc93286a43e5e4a47dbb948 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 Jan 2016 14:28:07 -0800 Subject: [PATCH 42/65] Relax scheduler dependency order + simplify some code The previous dependency order between TaskMetrics and accumulators in DAGScheduler is unnecessary. Removing this ordering constraint simplifies the code a little and makes the diff a little smaller. --- .../apache/spark/scheduler/DAGScheduler.scala | 43 ++++--------------- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../sql/execution/metric/SQLMetrics.scala | 2 +- 3 files changed, 11 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4642b8eeb3df..a5157793862b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1116,33 +1116,11 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - // Note: the following events must occur in this order: - // (1) Merge accumulator updates from this task - // (2) Reconstruct TaskMetrics - // (3) Post SparkListenerTaskEnd event - // (4) Post SparkListenerStageCompleted / SparkListenerJobEnd event - - // Merge accumulator updates from this task. We must do this before reconstructing - // TaskMetrics or else the listeners downstream will not receive the most recent values. - if (stageIdToStage.contains(stageId)) { - val stage = stageIdToStage(stageId) - // We should do this if the task either succeeded or failed with an exception. In the - // latter case executors may still send back accumulator updates, so we should try our - // best to collect the values. TODO: write a test. - val shouldUpdateAccums = event.reason match { - case Success => - task match { - // This being true means the job has not finished yet - case rt: ResultTask[_, _] => stage.asInstanceOf[ResultStage].activeJob.isDefined - case smt: ShuffleMapTask => true - } - case _: ExceptionFailure => true - case _ => false - } - if (shouldUpdateAccums) { - updateAccumulators(event) - } - } + outputCommitCoordinator.taskCompleted( + stageId, + taskId, + event.taskInfo.attemptNumber, // this is a task attempt number + event.reason) // Reconstruct task metrics. Note: this may be null if the task has failed. val taskMetrics: TaskMetrics = @@ -1158,12 +1136,6 @@ class DAGScheduler( null } - outputCommitCoordinator.taskCompleted( - stageId, - taskId, - event.taskInfo.attemptNumber, // this is a task attempt number - event.reason) - listenerBus.post(SparkListenerTaskEnd( stageId, task.stageAttemptId, taskType, event.reason, event.taskInfo, taskMetrics)) @@ -1184,6 +1156,7 @@ class DAGScheduler( resultStage.activeJob match { case Some(job) => if (!job.finished(rt.outputId)) { + updateAccumulators(event) job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -1210,6 +1183,7 @@ class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] + updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) @@ -1321,7 +1295,8 @@ class DAGScheduler( // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits case exceptionFailure: ExceptionFailure => - // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + // Tasks failed with exceptions might still have accumulator updates. TODO: write a test. + updateAccumulators(event) case TaskResultLost => // Do nothing here; the TaskScheduler handles these failures and resubmits the task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index e39eccf7f120..65ac5e514574 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -47,7 +47,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, - serializedData: ByteBuffer) { + serializedData: ByteBuffer): Unit = { getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 652206323aa1..8b257c3d85ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. * - * An implementation of SQLMetric should override `+=` and `add` to avoid boxing + * An implementation of SQLMetric should override `+=` and `add` to avoid boxing. */ private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( name: String, val param: SQLMetricParam[R, T]) From 596348f048584db7a5f8a90effec704258755c0e Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Wed, 13 Jan 2016 18:09:51 -0800 Subject: [PATCH 43/65] Fix HistoryServerSuite The problem was that the executor run time from a failed task was not included in the one aggregated across the stage. This is because the old test log did not have accumulator updates, but only TaskMetrics, in the ExceptionFailure class. It would seem that an easy fix would be to fallback to parsing TaskMetrics and get accumulator updates from there. However, this is insufficient because we could not reconstruct TaskMetrics without having previously registered them on the driver. This commit adds the functionality to reconstruct TaskMetrics without any prior state. This implicitly requires Spark to be able to reconstruct the Accumulator object from AccumulableInfo. This was difficult because this would mean that we need to keep track of the accumulator value type, which was not previously encoded anywhere. After this commit, we now use the correct type to de/serialize the values of AccumulableInfo's that represent the internal accumulators used by TaskMetrics. The two main benefits are: - ExceptionFailure no longer breaks compatibility in event logs - AccumulableInfo is type-safe in event logs for internal accums --- .../scala/org/apache/spark/Accumulators.scala | 107 ++++++++++++------ .../apache/spark/executor/TaskMetrics.scala | 48 ++++---- .../spark/scheduler/AccumulableInfo.scala | 1 + .../org/apache/spark/util/JsonProtocol.scala | 80 +++++++++++-- .../scala/org/apache/spark/util/Utils.scala | 1 - .../apache/spark/util/JsonProtocolSuite.scala | 65 +++++++---- 6 files changed, 214 insertions(+), 88 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 48f72dbc0aff..0f6b43867074 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -170,6 +170,12 @@ class Accumulable[R, T] private[spark] ( */ private[spark] def setValue(newValue: R): Unit = { value_ = newValue } + /** + * Set the accumulator's value. + * This is used to reconstruct [[org.apache.spark.executor.TaskMetrics]] from accumulator updates. + */ + private[spark] def setValueAny(newValue: Any): Unit = { setValue(newValue.asInstanceOf[R]) } + // Called by Java when deserializing an object private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() @@ -358,6 +364,10 @@ object AccumulatorParam { def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] } + // For the internal metric that records what blocks are updated in a particular task + private[spark] object UpdatedBlockStatusesAccumulatorParam + extends ListAccumulatorParam[(BlockId, BlockStatus)] + } // TODO: The multi-thread support in accumulators is kind of lame; check @@ -477,71 +487,102 @@ private[spark] object InternalAccumulator { // scalastyle:on + /** + * Create an internal [[Accumulator]] by name, which must begin with [[METRICS_PREFIX]]. + */ + def create(name: String): Accumulator[_] = { + assert(name.startsWith(METRICS_PREFIX), + s"internal accumulator name must start with '$METRICS_PREFIX': $name") + getParam(name) match { + case p @ LongAccumulatorParam => newMetric[Long](0L, name, p) + case p @ IntAccumulatorParam => newMetric[Int](0, name, p) + case p @ StringAccumulatorParam => newMetric[String]("", name, p) + case p @ UpdatedBlockStatusesAccumulatorParam => + newMetric[Seq[(BlockId, BlockStatus)]](Seq(), name, p) + case p => throw new IllegalArgumentException( + s"unsupported accumulator param '${p.getClass.getSimpleName}' for internal metrics.") + } + } + + /** + * Get the [[AccumulatorParam]] associated with the internal metric name, + * which must begin with [[METRICS_PREFIX]]. + */ + def getParam(name: String): AccumulatorParam[_] = { + assert(name.startsWith(METRICS_PREFIX), + s"internal accumulator name must start with '$METRICS_PREFIX': $name") + name match { + case UPDATED_BLOCK_STATUSES => UpdatedBlockStatusesAccumulatorParam + case shuffleRead.LOCAL_BLOCKS_FETCHED => IntAccumulatorParam + case shuffleRead.REMOTE_BLOCKS_FETCHED => IntAccumulatorParam + case input.READ_METHOD => StringAccumulatorParam + case output.WRITE_METHOD => StringAccumulatorParam + case _ => LongAccumulatorParam + } + } + /** * Accumulators for tracking internal metrics. */ def create(): Seq[Accumulator[_]] = { - Seq[Accumulator[_]]( - newLongMetric(EXECUTOR_DESERIALIZE_TIME), - newLongMetric(EXECUTOR_RUN_TIME), - newLongMetric(RESULT_SIZE), - newLongMetric(JVM_GC_TIME), - newLongMetric(RESULT_SERIALIZATION_TIME), - newLongMetric(MEMORY_BYTES_SPILLED), - newLongMetric(DISK_BYTES_SPILLED), - newLongMetric(PEAK_EXECUTION_MEMORY), - newMetric( - Seq.empty[(BlockId, BlockStatus)], - UPDATED_BLOCK_STATUSES, - new ListAccumulatorParam[(BlockId, BlockStatus)])) ++ + Seq[String]( + EXECUTOR_DESERIALIZE_TIME, + EXECUTOR_RUN_TIME, + RESULT_SIZE, + JVM_GC_TIME, + RESULT_SERIALIZATION_TIME, + MEMORY_BYTES_SPILLED, + DISK_BYTES_SPILLED, + PEAK_EXECUTION_MEMORY, + UPDATED_BLOCK_STATUSES).map(create) ++ createShuffleReadAccums() ++ createShuffleWriteAccums() ++ createInputAccums() ++ createOutputAccums() ++ - sys.props.get("spark.testing").map(_ => newLongMetric(TEST_ACCUM)).toSeq + sys.props.get("spark.testing").map(_ => create(TEST_ACCUM)).toSeq } /** * Accumulators for tracking shuffle read metrics. */ def createShuffleReadAccums(): Seq[Accumulator[_]] = { - Seq[Accumulator[_]]( - newIntMetric(shuffleRead.REMOTE_BLOCKS_FETCHED), - newIntMetric(shuffleRead.LOCAL_BLOCKS_FETCHED), - newLongMetric(shuffleRead.REMOTE_BYTES_READ), - newLongMetric(shuffleRead.LOCAL_BYTES_READ), - newLongMetric(shuffleRead.FETCH_WAIT_TIME), - newLongMetric(shuffleRead.RECORDS_READ)) + Seq[String]( + shuffleRead.REMOTE_BLOCKS_FETCHED, + shuffleRead.LOCAL_BLOCKS_FETCHED, + shuffleRead.REMOTE_BYTES_READ, + shuffleRead.LOCAL_BYTES_READ, + shuffleRead.FETCH_WAIT_TIME, + shuffleRead.RECORDS_READ).map(create) } /** * Accumulators for tracking shuffle write metrics. */ def createShuffleWriteAccums(): Seq[Accumulator[_]] = { - Seq[Accumulator[_]]( - newLongMetric(shuffleWrite.BYTES_WRITTEN), - newLongMetric(shuffleWrite.RECORDS_WRITTEN), - newLongMetric(shuffleWrite.WRITE_TIME)) + Seq[String]( + shuffleWrite.BYTES_WRITTEN, + shuffleWrite.RECORDS_WRITTEN, + shuffleWrite.WRITE_TIME).map(create) } /** * Accumulators for tracking input metrics. */ def createInputAccums(): Seq[Accumulator[_]] = { - Seq[Accumulator[_]]( - newStringMetric(input.READ_METHOD), - newLongMetric(input.BYTES_READ), - newLongMetric(input.RECORDS_READ)) + Seq[String]( + input.READ_METHOD, + input.BYTES_READ, + input.RECORDS_READ).map(create) } /** * Accumulators for tracking output metrics. */ private def createOutputAccums(): Seq[Accumulator[_]] = { - Seq[Accumulator[_]]( - newStringMetric(output.WRITE_METHOD), - newLongMetric(output.BYTES_WRITTEN), - newLongMetric(output.RECORDS_WRITTEN)) + Seq[String]( + output.WRITE_METHOD, + output.BYTES_WRITTEN, + output.RECORDS_WRITTEN).map(create) } /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 3d1bd826bcc9..17f4adf53a4c 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -324,7 +324,7 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser var (hasShuffleRead, hasShuffleWrite, hasInput, hasOutput) = (false, false, false, false) initialAccums .filter { a => a.localValue != a.zero } - .map { a => + .foreach { a => a.name.get match { case sr if sr.startsWith(SHUFFLE_READ_METRICS_PREFIX) => hasShuffleRead = true case sw if sw.startsWith(SHUFFLE_WRITE_METRICS_PREFIX) => hasShuffleWrite = true @@ -364,36 +364,42 @@ private[spark] object TaskMetrics extends Logging { } /** - * Construct a [[TaskMetrics]] object from a list of accumulator updates. + * Construct a [[TaskMetrics]] object from a list of accumulator updates, called on driver only. * - * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, - * we need the latter to post task end events to listeners, so we need to reconstruct the - * metrics on the driver. + * Executors only send accumulator updates back to the driver, not [[TaskMetrics]]. However, we + * need the latter to post task end events to listeners, so we need to reconstruct the metrics + * on the driver. * * Note: If the task failed, we may return null after attempting to reconstruct the * [[TaskMetrics]] in vain. */ def fromAccumulatorUpdates(taskId: Long, accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { - // Note: it is important that we copy the accumulators here since they are used across - // many tasks and we want to maintain a snapshot of their local task values when we post - // them to listeners downstream. Otherwise, when a new task comes in the listener may get - // a different value for an old task. - val accums = accumUpdates - .filter { info => info.update.isDefined } - .flatMap { info => - Accumulators.get(info.id).map(_.copy(info.update.get)).orElse { - logWarning(s"encountered unregistered accumulator ${info.id} " + - s"when reconstructing metrics for task $taskId.") - None - } - } // Initial accumulators are passed into the TaskMetrics constructor first because these // are required to be uniquely named. The rest of the accumulators from this task are // registered later because they need not satisfy this requirement. - val (initialAccums, otherAccums) = accums.partition { a => - a.name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) + val (initialAccumInfos, otherAccumInfos) = accumUpdates + .filter { info => info.update.isDefined } + .partition { info => + info.name != null && info.name.startsWith(InternalAccumulator.METRICS_PREFIX) + } + val initialAccums = initialAccumInfos.map { info => + val accum = InternalAccumulator.create(info.name) + accum.setValueAny(info.update.get) + accum + } + // We don't know the types of the rest of the accumulators, so we try to find the same ones + // that were previously registered here on the driver and make copies of them. It is important + // that we copy the accumulators here since they are used across many tasks and we want to + // maintain a snapshot of their local task values when we post them to listeners downstream. + val otherAccums = otherAccumInfos.flatMap { info => + val id = info.id + Accumulators.get(id).map(_.copy(info.update.get)).orElse { + logWarning(s"encountered unregistered accumulator $id " + + s"when reconstructing metrics for task $taskId.") + None + } } - val metrics = new TaskMetrics(initialAccums.map(_.asInstanceOf[Accumulator[_]])) + val metrics = new TaskMetrics(initialAccums) otherAccums.foreach(metrics.registerAccumulator) metrics } diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index e4315ffeeaa3..53c7b42ad0f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import org.apache.spark.annotation.DeveloperApi + /** * :: DeveloperApi :: * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. 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 16816b777986..8817418f6ed4 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -30,6 +30,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark._ +import org.apache.spark.AccumulatorParam._ import org.apache.spark.executor._ import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ @@ -284,13 +285,43 @@ private[spark] object JsonProtocol { } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { + val name = accumulableInfo.name ("ID" -> accumulableInfo.id) ~ - ("Name" -> accumulableInfo.name) ~ - ("Update" -> accumulableInfo.update.map(_.toString)) ~ - ("Value" -> accumulableInfo.value.map(_.toString)) ~ + ("Name" -> name) ~ + ("Update" -> accumulableInfo.update.map { v => accumValueToJson(name, v) }) ~ + ("Value" -> accumulableInfo.value.map { v => accumValueToJson(name, v) }) ~ ("Internal" -> accumulableInfo.internal) } + /** + * Serialize the value of an accumulator to JSON. + * + * For accmulators representing internal task metrics, this looks up the relevant + * [[AccumulatorParam]] to serialize the value accordingly. For all other accumulators, + * this will simply serialize the value as a string. + * + * The behavior here must match that of [[accumValueFromJson]]. TODO: add some tests. + */ + private def accumValueToJson(name: String, value: Any): JValue = { + if (name != null && name.startsWith(InternalAccumulator.METRICS_PREFIX)) { + (value, InternalAccumulator.getParam(name)) match { + case (v: Int, IntAccumulatorParam) => JInt(v) + case (v: Long, LongAccumulatorParam) => JInt(v) + case (v: String, StringAccumulatorParam) => JString(v) + case (v, UpdatedBlockStatusesAccumulatorParam) => + JArray(v.asInstanceOf[Seq[(BlockId, BlockStatus)]].toList.map { case (id, status) => + ("Block ID" -> id.toString) ~ + ("Status" -> blockStatusToJson(status)) + }) + case (v, p) => + throw new IllegalArgumentException(s"unexpected combination of accumulator value " + + s"type (${v.getClass.getName}) and accumulator param (${p.getClass.getName})") + } + } else { + JString(value.toString) + } + } + def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { val shuffleReadMetrics: JValue = taskMetrics.shuffleReadMetrics.map { rm => @@ -650,7 +681,7 @@ private[spark] object JsonProtocol { val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val accumulatedValues = (json \ "Accumulables").extractOpt[List[JValue]] match { - case Some(values) => values.map(accumulableInfoFromJson(_)) + case Some(values) => values.map(accumulableInfoFromJson) case None => Seq[AccumulableInfo]() } @@ -694,12 +725,42 @@ private[spark] object JsonProtocol { def accumulableInfoFromJson(json: JValue): AccumulableInfo = { val id = (json \ "ID").extract[Long] val name = (json \ "Name").extract[String] - val update = (json \ "Update").extractOpt[String] - val value = (json \ "Value").extractOpt[String] + val update = Utils.jsonOption(json \ "Update").map { v => accumValueFromJson(name, v) } + val value = Utils.jsonOption(json \ "Value").map { v => accumValueFromJson(name, v) } val internal = (json \ "Internal").extractOpt[Boolean].getOrElse(false) new AccumulableInfo(id, name, update, value, internal) } + /** + * Deserialize the value of an accumulator from JSON. + * + * For accmulators representing internal task metrics, this looks up the relevant + * [[AccumulatorParam]] to deserialize the value accordingly. For all other + * accumulators, this will simply deserialize the value as a string. + * + * The behavior here must match that of [[accumValueToJson]]. + */ + private def accumValueFromJson(name: String, value: JValue): Any = { + if (name != null && name.startsWith(InternalAccumulator.METRICS_PREFIX)) { + (value, InternalAccumulator.getParam(name)) match { + case (JInt(v), IntAccumulatorParam) => v.toInt + case (JInt(v), LongAccumulatorParam) => v.toLong + case (JString(v), StringAccumulatorParam) => v + case (JArray(v), UpdatedBlockStatusesAccumulatorParam) => + v.map { blockJson => + val id = BlockId((blockJson \ "Block ID").extract[String]) + val status = blockStatusFromJson(blockJson \ "Status") + (id, status) + } + case (v, p) => + throw new IllegalArgumentException(s"unexpected combination of accumulator " + + s"value in JSON ($v) and accumulator param (${p.getClass.getName})") + } + } else { + value.extract[String] + } + } + def taskMetricsFromJson(json: JValue): TaskMetrics = { if (json == JNothing) { return TaskMetrics.empty @@ -790,8 +851,11 @@ private[spark] object JsonProtocol { val description = (json \ "Description").extract[String] val stackTrace = stackTraceFromJson(json \ "Stack Trace") val fullStackTrace = (json \ "Full Stack Trace").extractOpt[String].orNull - val accumUpdates = (json \ "Accumulator Updates") - .extract[List[JValue]].map(accumulableInfoFromJson) + // Fallback on getting accumulator updates from TaskMetrics, which was logged in Spark 1.x + // TODO: add a test + val accumUpdates = Utils.jsonOption(json \ "Accumulator Updates") + .map(_.extract[List[JValue]].map(accumulableInfoFromJson)) + .getOrElse(taskMetricsFromJson(json \ "Metrics").accumulatorUpdates()) ExceptionFailure( className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost 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 077b87195aaf..9ecbffbf715c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1630,7 +1630,6 @@ private[spark] object Utils extends Logging { /** Return an option that translates JNothing to None */ def jsonOption(json: JValue): Option[JValue] = { - // TODO: this is useless; we can just use extractOpt, which looks cleaner. json match { case JNothing => None case value: JValue => Some(value) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 1f50251cb77b..65444215e92d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1681,7 +1681,6 @@ class JsonProtocolSuite extends SparkFunSuite { |} """ - // scalastyle:off private val executorMetricsUpdateJsonString = s""" |{ @@ -1696,109 +1695,125 @@ class JsonProtocolSuite extends SparkFunSuite { | { | "ID": 82, | "Name": "$EXECUTOR_DESERIALIZE_TIME", - | "Update": "300", + | "Update": 300, | "Internal": true | }, | { | "ID": 83, | "Name": "$EXECUTOR_RUN_TIME", - | "Update": "400", + | "Update": 400, | "Internal": true | }, | { | "ID": 84, | "Name": "$RESULT_SIZE", - | "Update": "500", + | "Update": 500, | "Internal": true | }, | { | "ID": 85, | "Name": "$JVM_GC_TIME", - | "Update": "600", + | "Update": 600, | "Internal": true | }, | { | "ID": 86, | "Name": "$RESULT_SERIALIZATION_TIME", - | "Update": "700", + | "Update": 700, | "Internal": true | }, | { | "ID": 87, | "Name": "$MEMORY_BYTES_SPILLED", - | "Update": "800", + | "Update": 800, | "Internal": true | }, | { | "ID": 88, | "Name": "$DISK_BYTES_SPILLED", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 89, | "Name": "$PEAK_EXECUTION_MEMORY", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 90, | "Name": "$UPDATED_BLOCK_STATUSES", - | "Update": "Vector((rdd_0_0,BlockStatus(StorageLevel(true, true, false, false, 2),0,0,0)))", + | "Update": [ + | { + | "BlockID": "rdd_0_0", + | "Status": { + | "StorageLevel": { + | "UseDisk": true, + | "UseMemory": true, + | "UseExternalBlockStore": false, + | "Deserialized": false, + | "Replication": 2 + | }, + | "MemorySize": 0, + | "ExternalBlockStoreSize": 0, + | "DiskSize": 0 + | } + | } + | ], | "Internal": true | }, | { | "ID": 91, | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 92, | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 93, | "Name": "${shuffleRead.REMOTE_BYTES_READ}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 94, | "Name": "${shuffleRead.LOCAL_BYTES_READ}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 95, | "Name": "${shuffleRead.FETCH_WAIT_TIME}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 96, | "Name": "${shuffleRead.RECORDS_READ}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 97, | "Name": "${shuffleWrite.BYTES_WRITTEN}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 98, | "Name": "${shuffleWrite.RECORDS_WRITTEN}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { | "ID": 99, | "Name": "${shuffleWrite.WRITE_TIME}", - | "Update": "0", + | "Update": 0, | "Internal": true | }, | { @@ -1810,13 +1825,13 @@ class JsonProtocolSuite extends SparkFunSuite { | { | "ID": 101, | "Name": "${input.BYTES_READ}", - | "Update": "2100", + | "Update": 2100, | "Internal": true | }, | { | "ID": 102, | "Name": "${input.RECORDS_READ}", - | "Update": "21", + | "Update": 21, | "Internal": true | }, | { @@ -1828,19 +1843,19 @@ class JsonProtocolSuite extends SparkFunSuite { | { | "ID": 104, | "Name": "${output.BYTES_WRITTEN}", - | "Update": "1200", + | "Update": 1200, | "Internal": true | }, | { | "ID": 105, | "Name": "${output.RECORDS_WRITTEN}", - | "Update": "12", + | "Update": 12, | "Internal": true | }, | { | "ID": 106, | "Name": "$TEST_ACCUM", - | "Update": "0", + | "Update": 0, | "Internal": true | } | ] @@ -1848,5 +1863,5 @@ class JsonProtocolSuite extends SparkFunSuite { | ] |} """.stripMargin - // scalastyle:on + } From 8b99071f1591abce3cb61651413747188fb47ec5 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Wed, 13 Jan 2016 18:27:06 -0800 Subject: [PATCH 44/65] Fix TaskMetricsSuite Do not create ShuffleReadMetrics when there is no shuffle read. --- .../org/apache/spark/executor/TaskMetrics.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 17f4adf53a4c..91713ff21bef 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -278,17 +278,16 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser * This is expected to be called on executor heartbeat and at the end of a task. */ private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { - val agg = _shuffleReadMetrics.getOrElse { + if (tempShuffleReadMetrics.nonEmpty) { val metrics = new ShuffleReadMetrics(initialAccumsMap.toMap) + metrics.setRemoteBlocksFetched(tempShuffleReadMetrics.map(_.remoteBlocksFetched).sum) + metrics.setLocalBlocksFetched(tempShuffleReadMetrics.map(_.localBlocksFetched).sum) + metrics.setFetchWaitTime(tempShuffleReadMetrics.map(_.fetchWaitTime).sum) + metrics.setRemoteBytesRead(tempShuffleReadMetrics.map(_.remoteBytesRead).sum) + metrics.setLocalBytesRead(tempShuffleReadMetrics.map(_.localBytesRead).sum) + metrics.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) _shuffleReadMetrics = Some(metrics) - metrics } - agg.setRemoteBlocksFetched(tempShuffleReadMetrics.map(_.remoteBlocksFetched).sum) - agg.setLocalBlocksFetched(tempShuffleReadMetrics.map(_.localBlocksFetched).sum) - agg.setFetchWaitTime(tempShuffleReadMetrics.map(_.fetchWaitTime).sum) - agg.setRemoteBytesRead(tempShuffleReadMetrics.map(_.remoteBytesRead).sum) - agg.setLocalBytesRead(tempShuffleReadMetrics.map(_.localBytesRead).sum) - agg.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) } From dd6d8ab72b3ddb025728365bb1e8116a7da67199 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Wed, 13 Jan 2016 18:59:29 -0800 Subject: [PATCH 45/65] Fix JsonProtocolSuite We were asserting the JSON serialized forms of the events, which may have out-of-order accumulators in StageInfo. Instead we should make it a SparkListenerEvent again and assert equality there. --- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../spark/scheduler/ReplayListenerSuite.scala | 8 +++++-- .../apache/spark/util/JsonProtocolSuite.scala | 23 +++++++++++-------- 3 files changed, 21 insertions(+), 12 deletions(-) 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 8817418f6ed4..09d953bd8f06 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -266,7 +266,7 @@ private[spark] object JsonProtocol { ("Completion Time" -> completionTime) ~ ("Failure Reason" -> failureReason) ~ ("Accumulables" -> JArray( - stageInfo.accumulables.values.map(accumulableInfoToJson).toList)) + stageInfo.accumulables.values.map(accumulableInfoToJson).toList)) } def taskInfoToJson(taskInfo: TaskInfo): JValue = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 761e82e6cf1c..35215c15ea80 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} /** * Test whether ReplayListenerBus replays events from logs correctly. @@ -131,7 +131,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(sc.eventLogger.isDefined) val originalEvents = sc.eventLogger.get.loggedEvents val replayedEvents = eventMonster.loggedEvents - originalEvents.zip(replayedEvents).foreach { case (e1, e2) => assert(e1 === e2) } + originalEvents.zip(replayedEvents).foreach { case (e1, e2) => + // Don't compare the JSON here because accumulators in StageInfo may be out of order + JsonProtocolSuite.assertEquals( + JsonProtocol.sparkEventFromJson(e1), JsonProtocol.sparkEventFromJson(e2)) + } } /** diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 65444215e92d..43cf0a8332d4 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,14 +32,7 @@ import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage._ class JsonProtocolSuite extends SparkFunSuite { - - import InternalAccumulator._ - - val jobSubmissionTime = 1421191042750L - val jobCompletionTime = 1421191296660L - - val executorAddedTime = 1421458410000L - val executorRemovedTime = 1421458922000L + import JsonProtocolSuite._ test("SparkListenerEvent") { val stageSubmitted = @@ -378,6 +371,18 @@ class JsonProtocolSuite extends SparkFunSuite { val oldInfo = JsonProtocol.accumulableInfoFromJson(oldJson) assert(false === oldInfo.internal) } +} + +// These helper methods are moved into an object so other test suites can reuse them. +// This extends SparkFunSuite only because we want to use scalatest asserts. +private[spark] object JsonProtocolSuite extends SparkFunSuite { + import InternalAccumulator._ + + val jobSubmissionTime = 1421191042750L + val jobCompletionTime = 1421191296660L + + val executorAddedTime = 1421458410000L + val executorRemovedTime = 1421458922000L /** -------------------------- * | Helper test running methods | @@ -444,7 +449,7 @@ class JsonProtocolSuite extends SparkFunSuite { | Util methods for comparing events | * --------------------------------- */ - private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { + private[spark] def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { (event1, event2) match { case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => assert(e1.properties === e2.properties) From 471429c04350b3d4f6c7d0ef296482a60f245ed4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 11:32:01 -0800 Subject: [PATCH 46/65] Fix TaskContextSuite + add some param docs --- .../main/scala/org/apache/spark/scheduler/ResultTask.scala | 4 ++++ .../scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 4 ++++ core/src/main/scala/org/apache/spark/scheduler/Task.scala | 4 ++++ .../scala/org/apache/spark/scheduler/TaskContextSuite.scala | 2 +- 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 074f159ab7a8..bcd34b367773 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD * See [[Task]] for more information. * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param taskBinary broadcasted version of the serialized RDD and the function to apply on each * partition of the given RDD. Once deserialized, the type should be * (RDD[T], (TaskContext, Iterator[T]) => U). @@ -37,6 +38,9 @@ import org.apache.spark.rdd.RDD * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). + * @param initialAccumulators initial set of accumulators to be used in this task for tracking + * internal metrics. Other accumulators will be registered later when + * they are deserialized on the executors. */ private[spark] class ResultTask[T, U]( stageId: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index c32ba314d73d..c399f62555ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -33,10 +33,14 @@ import org.apache.spark.shuffle.ShuffleWriter * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param taskBinary broadcast version of the RDD and the ShuffleDependency. Once deserialized, * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling + * @param initialAccumulators initial set of accumulators to be used in this task for tracking + * internal metrics. Other accumulators will be registered later when + * they are deserialized on the executors. */ private[spark] class ShuffleMapTask( stageId: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index fe4f619ab859..4a06b729537c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -41,7 +41,11 @@ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Uti * and divides the task output to multiple buckets (based on the task's partitioner). * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param partitionId index of the number in the RDD + * @param initialAccumulators initial set of accumulators to be used in this task for tracking + * internal metrics. Other accumulators will be registered later when + * they are deserialized on the executors. */ private[spark] abstract class Task[T]( val stageId: Int, 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 e5ec44a9f3b6..599ef1383792 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -58,7 +58,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val func = (c: TaskContext, i: Iterator[String]) => i.next() val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) val task = new ResultTask[String, String]( - 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, Seq.empty) + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, InternalAccumulator.create(sc)) intercept[RuntimeException] { task.run(0, 0, null) } From 28fae0f5846c6a29c7ed0d8e35d3aac22f11c314 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 11:49:31 -0800 Subject: [PATCH 47/65] Fix StagePageSuite --- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 8 +++----- .../scala/org/apache/spark/ui/StagePageSuite.scala | 11 ++++++----- 2 files changed, 9 insertions(+), 10 deletions(-) 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 fd730394d4d2..7936d0794157 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 @@ -885,11 +885,9 @@ private[ui] class TaskDataSource( val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val (_, taskExternalAccumulables) = - info.accumulables.partition(_.internal) - val externalAccumulableReadable = taskExternalAccumulables.map { acc => - StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update.get}") - } + val externalAccumulableReadable = info.accumulables + .filterNot(_.internal) + .map { acc => StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update.get}") } val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L) val maybeInput = metrics.flatMap(_.inputMetrics) 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 1e025bdf3916..b83ffa3282e4 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -31,6 +31,8 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener class StagePageSuite extends SparkFunSuite with LocalSparkContext { + private val peakExecutionMemory = 10 + test("peak execution memory only displayed if unsafe is enabled") { val unsafeConf = "spark.sql.unsafe.enabled" val conf = new SparkConf(false).set(unsafeConf, "true") @@ -52,7 +54,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val conf = new SparkConf(false).set(unsafeConf, "true") val html = renderStagePage(conf).toString().toLowerCase // verify min/25/50/75/max show task value not cumulative values - assert(html.contains("10.0 b" * 5)) + assert(html.contains(s"$peakExecutionMemory.0 b" * 5)) } /** @@ -79,14 +81,13 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { (1 to 2).foreach { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - val peakExecutionMemory = 10 - taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY, - Some(peakExecutionMemory.toString), Some((peakExecutionMemory * taskId).toString), true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) taskInfo.markSuccessful() + val taskMetrics = TaskMetrics.empty + taskMetrics.incPeakExecutionMemory(peakExecutionMemory) jobListener.onTaskEnd( - SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, TaskMetrics.empty)) + SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) } jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) page.render(request) From 1b027cb1669d9c411b120c2579443651fd5b1c7f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 12:18:12 -0800 Subject: [PATCH 48/65] Fix ExternalSorterSuite We never passed "countFailedValues" to AccumulableInfo. --- .../scala/org/apache/spark/Accumulators.scala | 7 ------- .../org/apache/spark/executor/TaskMetrics.scala | 3 ++- .../apache/spark/scheduler/AccumulableInfo.scala | 9 +++++++-- .../apache/spark/scheduler/DAGScheduler.scala | 8 ++++---- .../org/apache/spark/util/JsonProtocol.scala | 6 ++++-- .../spark/scheduler/DAGSchedulerSuite.scala | 16 ++++++++++------ .../spark/scheduler/TaskSetManagerSuite.scala | 4 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 3 ++- .../sql/execution/ui/SQLListenerSuite.scala | 3 ++- 9 files changed, 33 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 0f6b43867074..8450efb58d49 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -611,11 +611,4 @@ private[spark] object InternalAccumulator { new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) } - private def newIntMetric(name: String): Accumulator[Int] = - newMetric[Int](0, name, IntAccumulatorParam) - private def newLongMetric(name: String): Accumulator[Long] = - newMetric[Long](0L, name, LongAccumulatorParam) - private def newStringMetric(name: String): Accumulator[String] = - newMetric[String]("", name, StringAccumulatorParam) - } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 91713ff21bef..b3645aee03b7 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -314,7 +314,8 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser * Return a map from accumulator ID to the accumulator's latest value in this task. */ def accumulatorUpdates(): Seq[AccumulableInfo] = accums.map { a => - new AccumulableInfo(a.id, a.name.orNull, Some(a.localValue), None, a.isInternal) + new AccumulableInfo( + a.id, a.name.orNull, Some(a.localValue), None, a.isInternal, a.countFailedValues) } // If we are reconstructing this TaskMetrics on the driver, some metrics may already be set. diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 53c7b42ad0f9..716df6da601b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -41,5 +41,10 @@ case class AccumulableInfo private[spark] ( name: String, update: Option[Any], value: Option[Any], - internal: Boolean, - countFailedValues: Boolean = false) + private[spark] val internal: Boolean, + private[spark] val countFailedValues: Boolean) { + + def this(id: Long, name: String, update: Option[Any], value: Option[Any]) { + this(id, name, update, value, true /* internal */, false /* countFailedValues */) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a5157793862b..65bb804ca406 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1094,10 +1094,10 @@ class DAGScheduler( // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && partialValue != acc.zero) { val name = acc.name.get - stage.latestInfo.accumulables(id) = - new AccumulableInfo(id, name, None, Some(acc.value), acc.isInternal) - event.taskInfo.accumulables += - new AccumulableInfo(id, name, Some(partialValue), Some(acc.value), acc.isInternal) + stage.latestInfo.accumulables(id) = new AccumulableInfo( + id, name, None, Some(acc.value), acc.isInternal, acc.countFailedValues) + event.taskInfo.accumulables += new AccumulableInfo( + id, name, Some(partialValue), Some(acc.value), acc.isInternal, acc.countFailedValues) } } } catch { 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 09d953bd8f06..291e4c4cde76 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -290,7 +290,8 @@ private[spark] object JsonProtocol { ("Name" -> name) ~ ("Update" -> accumulableInfo.update.map { v => accumValueToJson(name, v) }) ~ ("Value" -> accumulableInfo.value.map { v => accumValueToJson(name, v) }) ~ - ("Internal" -> accumulableInfo.internal) + ("Internal" -> accumulableInfo.internal) ~ + ("Count Failed Values" -> accumulableInfo.countFailedValues) } /** @@ -728,7 +729,8 @@ private[spark] object JsonProtocol { val update = Utils.jsonOption(json \ "Update").map { v => accumValueFromJson(name, v) } val value = Utils.jsonOption(json \ "Value").map { v => accumValueFromJson(name, v) } val internal = (json \ "Internal").extractOpt[Boolean].getOrElse(false) - new AccumulableInfo(id, name, update, value, internal) + val countFailedValues = (json \ "Count Failed Values").extractOpt[Boolean].getOrElse(false) + new AccumulableInfo(id, name, update, value, internal, countFailedValues) } /** 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 ea9e0c95681e..c8642fe50d74 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -270,7 +270,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou taskSet.tasks(i), result._1, result._2, - Seq(new AccumulableInfo(accumId, "", Some(1), None, internal = false)))) + Seq(new AccumulableInfo( + accumId, "", Some(1), None, internal = false, countFailedValues = false)))) } } } @@ -347,9 +348,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } test("equals and hashCode AccumulableInfo") { - val accInfo1 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), true) - val accInfo2 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), false) - val accInfo3 = new AccumulableInfo(1, "Accumulable1", Some("delta1"), Some("val1"), false) + val accInfo1 = new AccumulableInfo( + 1, "Accumulable1", Some("delta1"), Some("val1"), internal = true, countFailedValues = false) + val accInfo2 = new AccumulableInfo( + 1, "Accumulable1", Some("delta1"), Some("val1"), internal = true, countFailedValues = false) + val accInfo3 = new AccumulableInfo( + 1, "Accumulable1", Some("delta1"), Some("val1"), internal = false, countFailedValues = false) assert(accInfo1 !== accInfo2) assert(accInfo2 === accInfo3) assert(accInfo2.hashCode() === accInfo3.hashCode()) @@ -1927,8 +1931,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou result: Any, extraAccumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { - val accumUpdates = task.initialAccumulators.map { - a => new AccumulableInfo(a.id, a.name.get, Some(a.zero), None, a.isInternal) + val accumUpdates = task.initialAccumulators.map { a => + new AccumulableInfo(a.id, a.name.get, Some(a.zero), None, a.isInternal, a.countFailedValues) } ++ extraAccumUpdates CompletionEvent(task, reason, result, accumUpdates, taskInfo) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 484632c0ea5c..6da2677be033 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -166,7 +166,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) val accumUpdates = taskSet.tasks.head.initialAccumulators.map { a => - new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal) + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal, a.countFailedValues) } // Offer a host with NO_PREF as the constraint, @@ -187,7 +187,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) val accumUpdatesByTask: Array[Seq[AccumulableInfo]] = taskSet.tasks.map { _.initialAccumulators.map { a => - new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal) + new AccumulableInfo(a.id, a.name.get, Some(0L), None, a.isInternal, a.countFailedValues) }} // First three offers should all find tasks diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 43cf0a8332d4..965e413f4c8c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -753,7 +753,8 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { } private def makeAccumulableInfo(id: Int, internal: Boolean = false): AccumulableInfo = - new AccumulableInfo(id, "Accumulable" + id, Some("delta" + id), Some("val" + id), internal) + new AccumulableInfo(id, s"Accumulable$id", Some(s"delta$id"), Some(s"val$id"), + internal, countFailedValues = false) /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 570d00778a96..0d3a8d9caeb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -72,7 +72,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { private def createTaskMetrics(accumulatorUpdates: Map[Long, Long]): TaskMetrics = { val metrics = mock(classOf[TaskMetrics]) when(metrics.accumulatorUpdates()).thenReturn(accumulatorUpdates.map { case (id, update) => - new AccumulableInfo(id, "", Some(new LongSQLMetricValue(update)), None, internal = true) + new AccumulableInfo(id, "", Some(new LongSQLMetricValue(update)), + value = None, internal = true, countFailedValues = true) }.toSeq) metrics } From 5d09221238a5dda4753f59d2980f769b5a074e7a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 13:21:10 -0800 Subject: [PATCH 49/65] Fix SQLMetricsSuite When we reconstruct the TaskMetrics we would copy the Accumulable without preserving the ID. However, the SQLListener downstream expects these internal metric accumulators to use the same IDs. The fix is to preserve the ID of the Accumulable when doing the copying. --- .../scala/org/apache/spark/Accumulators.scala | 46 +++++++++---------- .../apache/spark/executor/TaskMetrics.scala | 9 +++- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 8450efb58d49..8a651df30ad5 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -43,6 +43,7 @@ import org.apache.spark.util.Utils * * Operations are not thread-safe. * + * @param id ID of this accumulator; for internal use only. * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` * @param name human-readable name for use in Spark's web UI @@ -56,26 +57,36 @@ import org.apache.spark.util.Utils * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ -class Accumulable[R, T] private[spark] ( +class Accumulable[R, T] private ( + val id: Long, @transient initialValue: R, param: AccumulableParam[R, T], val name: Option[String], internal: Boolean, - val countFailedValues: Boolean = false) + val countFailedValues: Boolean) extends Serializable { private[spark] def this( - @transient initialValue: R, param: AccumulableParam[R, T], internal: Boolean) = { - this(initialValue, param, None, internal) + initialValue: R, + param: AccumulableParam[R, T], + name: Option[String], + internal: Boolean, + countFailedValues: Boolean) = { + this(Accumulators.newId(), initialValue, param, name, internal, countFailedValues) } - def this(@transient initialValue: R, param: AccumulableParam[R, T], name: Option[String]) = - this(initialValue, param, name, false) + private[spark] def this( + initialValue: R, + param: AccumulableParam[R, T], + name: Option[String], + internal: Boolean) = { + this(initialValue, param, name, internal, false /* countFailedValues */) + } - def this(@transient initialValue: R, param: AccumulableParam[R, T]) = - this(initialValue, param, None) + def this(initialValue: R, param: AccumulableParam[R, T], name: Option[String]) = + this(initialValue, param, name, false) - val id: Long = Accumulators.newId() + def this(initialValue: R, param: AccumulableParam[R, T]) = this(initialValue, param, None) @volatile @transient private var value_ : R = initialValue // Current value on driver val zero = param.zero(initialValue) // Zero value to be passed to executors @@ -96,12 +107,10 @@ class Accumulable[R, T] private[spark] ( private[spark] def isInternal: Boolean = internal /** - * Return a copy of this [[Accumulable]] with a new value. + * Return a copy of this [[Accumulable]]. */ - private[spark] def copy(newValue: Any): Accumulable[R, T] = { - val a = new Accumulable[R, T](initialValue, param, name, internal, countFailedValues) - a.setValue(newValue.asInstanceOf[R]) - a + private[spark] def copy(): Accumulable[R, T] = { + new Accumulable[R, T](id, initialValue, param, name, internal, countFailedValues) } /** @@ -291,15 +300,6 @@ class Accumulator[T] private[spark] ( override val countFailedValues: Boolean = false) extends Accumulable[T, T](initialValue, param, name, internal, countFailedValues) { - /** - * Return a copy of this [[Accumulator]] with a new value. - */ - private[spark] override def copy(newValue: Any): Accumulator[T] = { - val a = new Accumulator[T](initialValue, param, name, internal, countFailedValues) - a.setValue(newValue.asInstanceOf[T]) - a - } - def this(initialValue: T, param: AccumulatorParam[T], name: Option[String]) = { this(initialValue, param, name, false) } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index b3645aee03b7..6cca57f8f8e3 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -393,11 +393,16 @@ private[spark] object TaskMetrics extends Logging { // maintain a snapshot of their local task values when we post them to listeners downstream. val otherAccums = otherAccumInfos.flatMap { info => val id = info.id - Accumulators.get(id).map(_.copy(info.update.get)).orElse { + val acc = Accumulators.get(id).map { a => + val newAcc = a.copy() + newAcc.setValueAny(info.update.get) + newAcc + } + if (acc.isEmpty) { logWarning(s"encountered unregistered accumulator $id " + s"when reconstructing metrics for task $taskId.") - None } + acc } val metrics = new TaskMetrics(initialAccums) otherAccums.foreach(metrics.registerAccumulator) From 4bfbc7c297232c97b6249ee7a081c7016da11186 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 15:09:54 -0800 Subject: [PATCH 50/65] Fix PartitionBatchPruningSuite This test suite uses InMemoryColumnarTableScan, which internally creates some accumulators. There were two problems: (1) These accumulators were created before all tests, but after each test we clear them from the global map. Now we create these accumulators before *each* test instead. (2) These accumulators were not updated on the driver because we overwrite the registered accumulator every time we do a copy, e.g. when we reconstruct TaskMetrics from accumulator updates. We do a check in register now to avoid overwriting the original accumulator. --- .../scala/org/apache/spark/Accumulators.scala | 8 ++++- .../columnar/PartitionBatchPruningSuite.scala | 36 +++++++++++++------ 2 files changed, 33 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 8a651df30ad5..10098d5e882a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -395,9 +395,15 @@ private[spark] object Accumulators extends Logging { * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. * Note: if an accumulator is registered here, it should also be registered with the active * context cleaner for cleanup so as to avoid memory leaks. + * + * If an [[Accumulable]] with the same ID was already registered, do nothing instead of + * overwriting it. This can happen when we copy accumulators, e.g. when we reconstruct + * [[org.apache.spark.executor.TaskMetrics]] from accumulator updates. */ def register(a: Accumulable[_, _]): Unit = synchronized { - originals(a.id) = new WeakReference[Accumulable[_, _]](a) + if (!originals.contains(a.id)) { + originals(a.id) = new WeakReference[Accumulable[_, _]](a) + } } def remove(accId: Long) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index d762f7bfe914..0cd0dc487a28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -17,12 +17,19 @@ package org.apache.spark.sql.execution.columnar +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ -class PartitionBatchPruningSuite extends SparkFunSuite with SharedSQLContext { + +class PartitionBatchPruningSuite + extends SparkFunSuite + with BeforeAndAfterEach + with SharedSQLContext { + import testImplicits._ private lazy val originalColumnBatchSize = sqlContext.conf.columnBatchSize @@ -32,30 +39,39 @@ class PartitionBatchPruningSuite extends SparkFunSuite with SharedSQLContext { super.beforeAll() // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch sqlContext.setConf(SQLConf.COLUMN_BATCH_SIZE, 10) - - val pruningData = sparkContext.makeRDD((1 to 100).map { key => - val string = if (((key - 1) / 10) % 2 == 0) null else key.toString - TestData(key, string) - }, 5).toDF() - pruningData.registerTempTable("pruningData") - // Enable in-memory partition pruning sqlContext.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) // Enable in-memory table scan accumulators sqlContext.setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") - sqlContext.cacheTable("pruningData") } override protected def afterAll(): Unit = { try { sqlContext.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) sqlContext.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) - sqlContext.uncacheTable("pruningData") } finally { super.afterAll() } } + override protected def beforeEach(): Unit = { + super.beforeEach() + val pruningData = sparkContext.makeRDD((1 to 100).map { key => + val string = if (((key - 1) / 10) % 2 == 0) null else key.toString + TestData(key, string) + }, 5).toDF() + pruningData.registerTempTable("pruningData") + sqlContext.cacheTable("pruningData") + } + + override protected def afterEach(): Unit = { + try { + sqlContext.uncacheTable("pruningData") + } finally { + super.afterEach() + } + } + // Comparisons checkBatchPruning("SELECT key FROM pruningData WHERE key = 1", 1, 1)(Seq(1)) checkBatchPruning("SELECT key FROM pruningData WHERE 1 = key", 1, 1)(Seq(1)) From 0185b72fb25f276a4af9c5ec02612ae80cdd26ea Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 15:22:41 -0800 Subject: [PATCH 51/65] Clean up Accumulators object a little + some docs --- .../scala/org/apache/spark/Accumulators.scala | 29 +++++++++++-------- .../apache/spark/scheduler/DAGScheduler.scala | 6 ++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 10098d5e882a..83d444f0cc6b 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,6 +18,8 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} +import java.util.concurrent.atomic.AtomicLong +import javax.annotation.concurrent.GuardedBy import scala.collection.generic.Growable import scala.collection.mutable @@ -379,14 +381,16 @@ private[spark] object Accumulators extends Logging { * once the RDDs and user-code that reference them are cleaned up. * TODO: Don't use a global map; these should be tied to a SparkContext at the very least. */ + @GuardedBy("Accumulators") val originals = mutable.Map[Long, WeakReference[Accumulable[_, _]]]() - private var lastId: Long = 0 + private val nextId = new AtomicLong(0L) - def newId(): Long = synchronized { - lastId += 1 - lastId - } + /** + * Return a new globally unique ID for a new [[Accumulable]]. + * Note: Once you copy the [[Accumulable]] the ID is no longer unique. + */ + def newId(): Long = nextId.getAndIncrement /** * Register an accumulator created on the driver such that it can be used on the executors. @@ -406,16 +410,17 @@ private[spark] object Accumulators extends Logging { } } - def remove(accId: Long) { - synchronized { - originals.remove(accId) - } + /** + * Unregister the [[Accumulable]] with the given ID, if any. + */ + def remove(accId: Long): Unit = synchronized { + originals.remove(accId) } /** - * Return the accumulator registered with the given ID, if any. + * Return the [[Accumulable]] registered with the given ID, if any. */ - def get(id: Long): Option[Accumulable[_, _]] = { + def get(id: Long): Option[Accumulable[_, _]] = synchronized { originals.get(id).map { weakRef => // Since we are storing weak references, we must check whether the underlying data is valid. weakRef.get match { @@ -427,7 +432,7 @@ private[spark] object Accumulators extends Logging { } /** - * Clear all registered accumulators; for testing only. + * Clear all registered [[Accumulable]]s; for testing only. */ def clear(): Unit = synchronized { originals.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 65bb804ca406..cf696f80f818 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1075,6 +1075,12 @@ class DAGScheduler( /** * Merge local values from a task into the corresponding accumulators previously registered * here on the driver. + * + * Although accumulators themselves are not thread-safe, this method is called only from one + * thread, the one that runs the scheduling loop. This means we only handle one task + * completion event at a time so we don't need to worry about locking the accumulators. + * This still doesn't stop the caller from updating the accumulator outside the scheduler, + * but that's not our problem since there's nothing we can do about that. */ private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task From b31862a089b20e516c77b558917c4210cdb31cfd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 15:25:03 -0800 Subject: [PATCH 52/65] Fix DAGSchedulerSuite, trivial omission --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c8642fe50d74..b5532c0e2d8f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -351,7 +351,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou val accInfo1 = new AccumulableInfo( 1, "Accumulable1", Some("delta1"), Some("val1"), internal = true, countFailedValues = false) val accInfo2 = new AccumulableInfo( - 1, "Accumulable1", Some("delta1"), Some("val1"), internal = true, countFailedValues = false) + 1, "Accumulable1", Some("delta1"), Some("val1"), internal = false, countFailedValues = false) val accInfo3 = new AccumulableInfo( 1, "Accumulable1", Some("delta1"), Some("val1"), internal = false, countFailedValues = false) assert(accInfo1 !== accInfo2) From 3dcc4e16acd0c2ceaa264ad23abcf067935d6a8f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 16:30:57 -0800 Subject: [PATCH 53/65] Fix style + default --- .../src/main/scala/org/apache/spark/Accumulators.scala | 10 ++++++++-- .../org/apache/spark/scheduler/AccumulableInfo.scala | 2 +- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 83d444f0cc6b..c67246a95f38 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -110,6 +110,10 @@ class Accumulable[R, T] private ( /** * Return a copy of this [[Accumulable]]. + * + * The copy will have the same ID as the original and will not be registered with + * [[Accumulators]] again. This method exists so that the caller can avoid passing the + * same mutable instance around. */ private[spark] def copy(): Accumulable[R, T] = { new Accumulable[R, T](id, initialValue, param, name, internal, countFailedValues) @@ -193,6 +197,8 @@ class Accumulable[R, T] private ( value_ = zero deserialized = true // Automatically register the accumulator when it is deserialized with the task closure. + // This is for external accumulators and internal ones that do not represent task level + // metrics, e.g. internal SQL metrics, which are per-operator. val taskContext = TaskContext.get() if (taskContext != null) { taskContext.registerAccumulator(this) @@ -387,7 +393,7 @@ private[spark] object Accumulators extends Logging { private val nextId = new AtomicLong(0L) /** - * Return a new globally unique ID for a new [[Accumulable]]. + * Return a globally unique ID for a new [[Accumulable]]. * Note: Once you copy the [[Accumulable]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement @@ -401,7 +407,7 @@ private[spark] object Accumulators extends Logging { * context cleaner for cleanup so as to avoid memory leaks. * * If an [[Accumulable]] with the same ID was already registered, do nothing instead of - * overwriting it. This can happen when we copy accumulators, e.g. when we reconstruct + * overwriting it. This happens when we copy accumulators, e.g. when we reconstruct * [[org.apache.spark.executor.TaskMetrics]] from accumulator updates. */ def register(a: Accumulable[_, _]): Unit = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 716df6da601b..8008dfcf2516 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -45,6 +45,6 @@ case class AccumulableInfo private[spark] ( private[spark] val countFailedValues: Boolean) { def this(id: Long, name: String, update: Option[Any], value: Option[Any]) { - this(id, name, update, value, true /* internal */, false /* countFailedValues */) + this(id, name, update, value, false, false) } } From 36a5ca826b19d9f42b943f7f4d8f510d79be679e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 16:44:34 -0800 Subject: [PATCH 54/65] Split Accumulators.scala into 3 files ... mainly for readability. --- .../org/apache/spark/AccumulableParam.scala | 150 +++++++++ .../scala/org/apache/spark/Accumulators.scala | 309 +----------------- .../apache/spark/InternalAccumulator.scala | 209 ++++++++++++ 3 files changed, 361 insertions(+), 307 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/AccumulableParam.scala create mode 100644 core/src/main/scala/org/apache/spark/InternalAccumulator.scala diff --git a/core/src/main/scala/org/apache/spark/AccumulableParam.scala b/core/src/main/scala/org/apache/spark/AccumulableParam.scala new file mode 100644 index 000000000000..7c6defe4166a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/AccumulableParam.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.Serializable + +import scala.collection.generic.Growable +import scala.reflect.ClassTag + +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage.{BlockStatus, BlockId} + + +/** + * Helper object defining how to accumulate values of a particular type. An implicit + * AccumulableParam needs to be available when you create [[Accumulable]]s of a specific type. + * + * @tparam R the full accumulated data (result type) + * @tparam T partial data that can be added in + */ +trait AccumulableParam[R, T] extends Serializable { + /** + * Add additional data to the accumulator value. Is allowed to modify and return `r` + * for efficiency (to avoid allocating objects). + * + * @param r the current value of the accumulator + * @param t the data to be added to the accumulator + * @return the new value of the accumulator + */ + def addAccumulator(r: R, t: T): R + + /** + * Merge two accumulated values together. Is allowed to modify and return the first value + * for efficiency (to avoid allocating objects). + * + * @param r1 one set of accumulated data + * @param r2 another set of accumulated data + * @return both data sets merged together + */ + def addInPlace(r1: R, r2: R): R + + /** + * Return the "zero" (identity) value for an accumulator type, given its initial value. For + * example, if R was a vector of N dimensions, this would return a vector of N zeroes. + */ + def zero(initialValue: R): R +} + + +private[spark] class +GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] + extends AccumulableParam[R, T] { + + def addAccumulator(growable: R, elem: T): R = { + growable += elem + growable + } + + def addInPlace(t1: R, t2: R): R = { + t1 ++= t2 + t1 + } + + def zero(initialValue: R): R = { + // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. + // Instead we'll serialize it to a buffer and load it back. + val ser = new JavaSerializer(new SparkConf(false)).newInstance() + val copy = ser.deserialize[R](ser.serialize(initialValue)) + copy.clear() // In case it contained stuff + copy + } +} + + +/** + * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add + * in is the same type as the accumulated value. An implicit AccumulatorParam object needs to be + * available when you create Accumulators of a specific type. + * + * @tparam T type of value to accumulate + */ +trait AccumulatorParam[T] extends AccumulableParam[T, T] { + def addAccumulator(t1: T, t2: T): T = { + addInPlace(t1, t2) + } +} + + +object AccumulatorParam { + + // The following implicit objects were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, as there are duplicate codes in SparkContext for backward + // compatibility, please update them accordingly if you modify the following implicit objects. + + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 + def zero(initialValue: Double): Double = 0.0 + } + + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 + def zero(initialValue: Int): Int = 0 + } + + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L + } + + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f + } + + // Note: when merging values, this param just adopts the newer value. This is used only + // internally for things that shouldn't really be accumulated across tasks, like input + // read method, which should be the same across all tasks in the same stage. + private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { + def addInPlace(t1: String, t2: String): String = t2 + def zero(initialValue: String): String = "" + } + + // Note: this is expensive as it makes a copy of the list every time the caller adds an item. + // A better way to use this is to first accumulate the values yourself then them all at once. + private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { + def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 + def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] + } + + // For the internal metric that records what blocks are updated in a particular task + private[spark] object UpdatedBlockStatusesAccumulatorParam + extends ListAccumulatorParam[(BlockId, BlockStatus)] + +} diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index c67246a95f38..9414394cc9fa 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,15 +21,12 @@ import java.io.{ObjectInputStream, Serializable} import java.util.concurrent.atomic.AtomicLong import javax.annotation.concurrent.GuardedBy -import scala.collection.generic.Growable import scala.collection.mutable import scala.ref.WeakReference -import scala.reflect.ClassTag -import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.Utils + /** * A data type that can be accumulated, ie has an commutative and associative "add" operation, * but where the result type, `R`, may be different from the element type being added, `T`. @@ -208,64 +205,6 @@ class Accumulable[R, T] private ( override def toString: String = if (value_ == null) "null" else value_.toString } -/** - * Helper object defining how to accumulate values of a particular type. An implicit - * AccumulableParam needs to be available when you create [[Accumulable]]s of a specific type. - * - * @tparam R the full accumulated data (result type) - * @tparam T partial data that can be added in - */ -trait AccumulableParam[R, T] extends Serializable { - /** - * Add additional data to the accumulator value. Is allowed to modify and return `r` - * for efficiency (to avoid allocating objects). - * - * @param r the current value of the accumulator - * @param t the data to be added to the accumulator - * @return the new value of the accumulator - */ - def addAccumulator(r: R, t: T): R - - /** - * Merge two accumulated values together. Is allowed to modify and return the first value - * for efficiency (to avoid allocating objects). - * - * @param r1 one set of accumulated data - * @param r2 another set of accumulated data - * @return both data sets merged together - */ - def addInPlace(r1: R, r2: R): R - - /** - * Return the "zero" (identity) value for an accumulator type, given its initial value. For - * example, if R was a vector of N dimensions, this would return a vector of N zeroes. - */ - def zero(initialValue: R): R -} - -private[spark] class -GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] - extends AccumulableParam[R, T] { - - def addAccumulator(growable: R, elem: T): R = { - growable += elem - growable - } - - def addInPlace(t1: R, t2: R): R = { - t1 ++= t2 - t1 - } - - def zero(initialValue: R): R = { - // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. - // Instead we'll serialize it to a buffer and load it back. - val ser = new JavaSerializer(new SparkConf(false)).newInstance() - val copy = ser.deserialize[R](ser.serialize(initialValue)) - copy.clear() // In case it contained stuff - copy - } -} /** * A simpler value of [[Accumulable]] where the result type being accumulated is the same @@ -317,66 +256,6 @@ class Accumulator[T] private[spark] ( } } -/** - * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add - * in is the same type as the accumulated value. An implicit AccumulatorParam object needs to be - * available when you create Accumulators of a specific type. - * - * @tparam T type of value to accumulate - */ -trait AccumulatorParam[T] extends AccumulableParam[T, T] { - def addAccumulator(t1: T, t2: T): T = { - addInPlace(t1, t2) - } -} - -object AccumulatorParam { - - // The following implicit objects were in SparkContext before 1.2 and users had to - // `import SparkContext._` to enable them. Now we move them here to make the compiler find - // them automatically. However, as there are duplicate codes in SparkContext for backward - // compatibility, please update them accordingly if you modify the following implicit objects. - - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { - def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double): Double = 0.0 - } - - implicit object IntAccumulatorParam extends AccumulatorParam[Int] { - def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int): Int = 0 - } - - implicit object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long): Long = t1 + t2 - def zero(initialValue: Long): Long = 0L - } - - implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float): Float = t1 + t2 - def zero(initialValue: Float): Float = 0f - } - - // Note: when merging values, this param just adopts the newer value. This is used only - // internally for things that shouldn't really be accumulated across tasks, like input - // read method, which should be the same across all tasks in the same stage. - private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { - def addInPlace(t1: String, t2: String): String = t2 - def zero(initialValue: String): String = "" - } - - // Note: this is expensive as it makes a copy of the list every time the caller adds an item. - // A better way to use this is to first accumulate the values yourself then them all at once. - private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { - def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 - def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] - } - - // For the internal metric that records what blocks are updated in a particular task - private[spark] object UpdatedBlockStatusesAccumulatorParam - extends ListAccumulatorParam[(BlockId, BlockStatus)] - -} // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right @@ -399,7 +278,7 @@ private[spark] object Accumulators extends Logging { def newId(): Long = nextId.getAndIncrement /** - * Register an accumulator created on the driver such that it can be used on the executors. + * Register an [[Accumulable]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. @@ -445,187 +324,3 @@ private[spark] object Accumulators extends Logging { } } - -private[spark] object InternalAccumulator { - - import AccumulatorParam._ - - // Prefixes used in names of internal task level metrics - val METRICS_PREFIX = "internal.metrics." - val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." - val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." - val OUTPUT_METRICS_PREFIX = METRICS_PREFIX + "output." - val INPUT_METRICS_PREFIX = METRICS_PREFIX + "input." - - // Names of internal task level metrics - val EXECUTOR_DESERIALIZE_TIME = METRICS_PREFIX + "executorDeserializeTime" - val EXECUTOR_RUN_TIME = METRICS_PREFIX + "executorRunTime" - val RESULT_SIZE = METRICS_PREFIX + "resultSize" - val JVM_GC_TIME = METRICS_PREFIX + "jvmGCTime" - val RESULT_SERIALIZATION_TIME = METRICS_PREFIX + "resultSerializationTime" - val MEMORY_BYTES_SPILLED = METRICS_PREFIX + "memoryBytesSpilled" - val DISK_BYTES_SPILLED = METRICS_PREFIX + "diskBytesSpilled" - val PEAK_EXECUTION_MEMORY = METRICS_PREFIX + "peakExecutionMemory" - val UPDATED_BLOCK_STATUSES = METRICS_PREFIX + "updatedBlockStatuses" - val TEST_ACCUM = METRICS_PREFIX + "testAccumulator" - - // scalastyle:off - - // Names of shuffle read metrics - object shuffleRead { - val REMOTE_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "remoteBlocksFetched" - val LOCAL_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "localBlocksFetched" - val REMOTE_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesRead" - val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead" - val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime" - val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead" - } - - // Names of shuffle write metrics - object shuffleWrite { - val BYTES_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "bytesWritten" - val RECORDS_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "recordsWritten" - val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" - } - - // Names of output metrics - object output { - val WRITE_METHOD = OUTPUT_METRICS_PREFIX + "writeMethod" - val BYTES_WRITTEN = OUTPUT_METRICS_PREFIX + "bytesWritten" - val RECORDS_WRITTEN = OUTPUT_METRICS_PREFIX + "recordsWritten" - } - - // Names of input metrics - object input { - val READ_METHOD = INPUT_METRICS_PREFIX + "readMethod" - val BYTES_READ = INPUT_METRICS_PREFIX + "bytesRead" - val RECORDS_READ = INPUT_METRICS_PREFIX + "recordsRead" - } - - // scalastyle:on - - /** - * Create an internal [[Accumulator]] by name, which must begin with [[METRICS_PREFIX]]. - */ - def create(name: String): Accumulator[_] = { - assert(name.startsWith(METRICS_PREFIX), - s"internal accumulator name must start with '$METRICS_PREFIX': $name") - getParam(name) match { - case p @ LongAccumulatorParam => newMetric[Long](0L, name, p) - case p @ IntAccumulatorParam => newMetric[Int](0, name, p) - case p @ StringAccumulatorParam => newMetric[String]("", name, p) - case p @ UpdatedBlockStatusesAccumulatorParam => - newMetric[Seq[(BlockId, BlockStatus)]](Seq(), name, p) - case p => throw new IllegalArgumentException( - s"unsupported accumulator param '${p.getClass.getSimpleName}' for internal metrics.") - } - } - - /** - * Get the [[AccumulatorParam]] associated with the internal metric name, - * which must begin with [[METRICS_PREFIX]]. - */ - def getParam(name: String): AccumulatorParam[_] = { - assert(name.startsWith(METRICS_PREFIX), - s"internal accumulator name must start with '$METRICS_PREFIX': $name") - name match { - case UPDATED_BLOCK_STATUSES => UpdatedBlockStatusesAccumulatorParam - case shuffleRead.LOCAL_BLOCKS_FETCHED => IntAccumulatorParam - case shuffleRead.REMOTE_BLOCKS_FETCHED => IntAccumulatorParam - case input.READ_METHOD => StringAccumulatorParam - case output.WRITE_METHOD => StringAccumulatorParam - case _ => LongAccumulatorParam - } - } - - /** - * Accumulators for tracking internal metrics. - */ - def create(): Seq[Accumulator[_]] = { - Seq[String]( - EXECUTOR_DESERIALIZE_TIME, - EXECUTOR_RUN_TIME, - RESULT_SIZE, - JVM_GC_TIME, - RESULT_SERIALIZATION_TIME, - MEMORY_BYTES_SPILLED, - DISK_BYTES_SPILLED, - PEAK_EXECUTION_MEMORY, - UPDATED_BLOCK_STATUSES).map(create) ++ - createShuffleReadAccums() ++ - createShuffleWriteAccums() ++ - createInputAccums() ++ - createOutputAccums() ++ - sys.props.get("spark.testing").map(_ => create(TEST_ACCUM)).toSeq - } - - /** - * Accumulators for tracking shuffle read metrics. - */ - def createShuffleReadAccums(): Seq[Accumulator[_]] = { - Seq[String]( - shuffleRead.REMOTE_BLOCKS_FETCHED, - shuffleRead.LOCAL_BLOCKS_FETCHED, - shuffleRead.REMOTE_BYTES_READ, - shuffleRead.LOCAL_BYTES_READ, - shuffleRead.FETCH_WAIT_TIME, - shuffleRead.RECORDS_READ).map(create) - } - - /** - * Accumulators for tracking shuffle write metrics. - */ - def createShuffleWriteAccums(): Seq[Accumulator[_]] = { - Seq[String]( - shuffleWrite.BYTES_WRITTEN, - shuffleWrite.RECORDS_WRITTEN, - shuffleWrite.WRITE_TIME).map(create) - } - - /** - * Accumulators for tracking input metrics. - */ - def createInputAccums(): Seq[Accumulator[_]] = { - Seq[String]( - input.READ_METHOD, - input.BYTES_READ, - input.RECORDS_READ).map(create) - } - - /** - * Accumulators for tracking output metrics. - */ - private def createOutputAccums(): Seq[Accumulator[_]] = { - Seq[String]( - output.WRITE_METHOD, - output.BYTES_WRITTEN, - output.RECORDS_WRITTEN).map(create) - } - - /** - * Accumulators for tracking internal metrics. - * - * These accumulators are created with the stage such that all tasks in the stage will - * add to the same set of accumulators. We do this to report the distribution of accumulator - * values across all tasks within each stage. - */ - def create(sc: SparkContext): Seq[Accumulator[_]] = { - val accums = create() - accums.foreach { accum => - Accumulators.register(accum) - sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) - } - accums - } - - /** - * Create a new accumulator representing an internal task metric. - */ - private def newMetric[T]( - initialValue: T, - name: String, - param: AccumulatorParam[T]): Accumulator[T] = { - new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) - } - -} diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala new file mode 100644 index 000000000000..7904e081e4fd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.apache.spark.storage.{BlockStatus, BlockId} + + +/** + * A collection of fields and methods concerned with internal accumulators that represent + * task level metrics. + */ +private[spark] object InternalAccumulator { + + import AccumulatorParam._ + + // Prefixes used in names of internal task level metrics + val METRICS_PREFIX = "internal.metrics." + val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." + val SHUFFLE_WRITE_METRICS_PREFIX = METRICS_PREFIX + "shuffle.write." + val OUTPUT_METRICS_PREFIX = METRICS_PREFIX + "output." + val INPUT_METRICS_PREFIX = METRICS_PREFIX + "input." + + // Names of internal task level metrics + val EXECUTOR_DESERIALIZE_TIME = METRICS_PREFIX + "executorDeserializeTime" + val EXECUTOR_RUN_TIME = METRICS_PREFIX + "executorRunTime" + val RESULT_SIZE = METRICS_PREFIX + "resultSize" + val JVM_GC_TIME = METRICS_PREFIX + "jvmGCTime" + val RESULT_SERIALIZATION_TIME = METRICS_PREFIX + "resultSerializationTime" + val MEMORY_BYTES_SPILLED = METRICS_PREFIX + "memoryBytesSpilled" + val DISK_BYTES_SPILLED = METRICS_PREFIX + "diskBytesSpilled" + val PEAK_EXECUTION_MEMORY = METRICS_PREFIX + "peakExecutionMemory" + val UPDATED_BLOCK_STATUSES = METRICS_PREFIX + "updatedBlockStatuses" + val TEST_ACCUM = METRICS_PREFIX + "testAccumulator" + + // scalastyle:off + + // Names of shuffle read metrics + object shuffleRead { + val REMOTE_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "localBlocksFetched" + val REMOTE_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesRead" + val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead" + val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime" + val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead" + } + + // Names of shuffle write metrics + object shuffleWrite { + val BYTES_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "bytesWritten" + val RECORDS_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "recordsWritten" + val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" + } + + // Names of output metrics + object output { + val WRITE_METHOD = OUTPUT_METRICS_PREFIX + "writeMethod" + val BYTES_WRITTEN = OUTPUT_METRICS_PREFIX + "bytesWritten" + val RECORDS_WRITTEN = OUTPUT_METRICS_PREFIX + "recordsWritten" + } + + // Names of input metrics + object input { + val READ_METHOD = INPUT_METRICS_PREFIX + "readMethod" + val BYTES_READ = INPUT_METRICS_PREFIX + "bytesRead" + val RECORDS_READ = INPUT_METRICS_PREFIX + "recordsRead" + } + + // scalastyle:on + + /** + * Create an internal [[Accumulator]] by name, which must begin with [[METRICS_PREFIX]]. + */ + def create(name: String): Accumulator[_] = { + assert(name.startsWith(METRICS_PREFIX), + s"internal accumulator name must start with '$METRICS_PREFIX': $name") + getParam(name) match { + case p @ LongAccumulatorParam => newMetric[Long](0L, name, p) + case p @ IntAccumulatorParam => newMetric[Int](0, name, p) + case p @ StringAccumulatorParam => newMetric[String]("", name, p) + case p @ UpdatedBlockStatusesAccumulatorParam => + newMetric[Seq[(BlockId, BlockStatus)]](Seq(), name, p) + case p => throw new IllegalArgumentException( + s"unsupported accumulator param '${p.getClass.getSimpleName}' for internal metrics.") + } + } + + /** + * Get the [[AccumulatorParam]] associated with the internal metric name, + * which must begin with [[METRICS_PREFIX]]. + */ + def getParam(name: String): AccumulatorParam[_] = { + assert(name.startsWith(METRICS_PREFIX), + s"internal accumulator name must start with '$METRICS_PREFIX': $name") + name match { + case UPDATED_BLOCK_STATUSES => UpdatedBlockStatusesAccumulatorParam + case shuffleRead.LOCAL_BLOCKS_FETCHED => IntAccumulatorParam + case shuffleRead.REMOTE_BLOCKS_FETCHED => IntAccumulatorParam + case input.READ_METHOD => StringAccumulatorParam + case output.WRITE_METHOD => StringAccumulatorParam + case _ => LongAccumulatorParam + } + } + + /** + * Accumulators for tracking internal metrics. + */ + def create(): Seq[Accumulator[_]] = { + Seq[String]( + EXECUTOR_DESERIALIZE_TIME, + EXECUTOR_RUN_TIME, + RESULT_SIZE, + JVM_GC_TIME, + RESULT_SERIALIZATION_TIME, + MEMORY_BYTES_SPILLED, + DISK_BYTES_SPILLED, + PEAK_EXECUTION_MEMORY, + UPDATED_BLOCK_STATUSES).map(create) ++ + createShuffleReadAccums() ++ + createShuffleWriteAccums() ++ + createInputAccums() ++ + createOutputAccums() ++ + sys.props.get("spark.testing").map(_ => create(TEST_ACCUM)).toSeq + } + + /** + * Accumulators for tracking shuffle read metrics. + */ + def createShuffleReadAccums(): Seq[Accumulator[_]] = { + Seq[String]( + shuffleRead.REMOTE_BLOCKS_FETCHED, + shuffleRead.LOCAL_BLOCKS_FETCHED, + shuffleRead.REMOTE_BYTES_READ, + shuffleRead.LOCAL_BYTES_READ, + shuffleRead.FETCH_WAIT_TIME, + shuffleRead.RECORDS_READ).map(create) + } + + /** + * Accumulators for tracking shuffle write metrics. + */ + def createShuffleWriteAccums(): Seq[Accumulator[_]] = { + Seq[String]( + shuffleWrite.BYTES_WRITTEN, + shuffleWrite.RECORDS_WRITTEN, + shuffleWrite.WRITE_TIME).map(create) + } + + /** + * Accumulators for tracking input metrics. + */ + def createInputAccums(): Seq[Accumulator[_]] = { + Seq[String]( + input.READ_METHOD, + input.BYTES_READ, + input.RECORDS_READ).map(create) + } + + /** + * Accumulators for tracking output metrics. + */ + private def createOutputAccums(): Seq[Accumulator[_]] = { + Seq[String]( + output.WRITE_METHOD, + output.BYTES_WRITTEN, + output.RECORDS_WRITTEN).map(create) + } + + /** + * Accumulators for tracking internal metrics. + * + * These accumulators are created with the stage such that all tasks in the stage will + * add to the same set of accumulators. We do this to report the distribution of accumulator + * values across all tasks within each stage. + */ + def create(sc: SparkContext): Seq[Accumulator[_]] = { + val accums = create() + accums.foreach { accum => + Accumulators.register(accum) + sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) + } + accums + } + + /** + * Create a new accumulator representing an internal task metric. + */ + private def newMetric[T]( + initialValue: T, + name: String, + param: AccumulatorParam[T]): Accumulator[T] = { + new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) + } + +} From fe451e5d516413347ed7bd7ebd94eb25088de60e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 17:26:02 -0800 Subject: [PATCH 55/65] Fix MiMa --- project/MimaExcludes.scala | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bdb2dfaed62d..74eac2681029 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -135,6 +135,18 @@ object MimaExcludes { ) ++ Seq( // SPARK-12510 Refactor ActorReceiver to support Java ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-10620 Migrate TaskMetrics to accumulators + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.internalMetricsToAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.collectInternalAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.collectAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulable.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulator.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this") ) ++ Seq( // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), @@ -147,16 +159,6 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") - ) ++ Seq( - // SPARK-10620 Migrate TaskMetrics to accumulators - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.TaskContext.internalMetricsToAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.TaskContext.collectInternalAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.TaskContext.collectAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulable.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.this") ) case v if v.startsWith("1.6") => Seq( From f90006852d3777e189ef36b125152f41a893aa84 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 17:44:02 -0800 Subject: [PATCH 56/65] Minor cleanups --- .../apache/spark/executor/TaskMetrics.scala | 27 ++++++++++--------- .../columnar/PartitionBatchPruningSuite.scala | 2 ++ 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 6cca57f8f8e3..36b5337a0b82 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -42,8 +42,9 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * be sent to the driver directly. * * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. - * Each accumulator in this initial set must be named and marked as internal. - * Additional accumulators registered here have no such requirements. + * Each accumulator in this initial set must be uniquely named and marked + * as internal. Additional accumulators registered later need not satisfy + * these requirements. */ @DeveloperApi class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Serializable { @@ -138,17 +139,19 @@ class TaskMetrics private[spark](initialAccums: Seq[Accumulator[_]]) extends Ser */ def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue - private[spark] def setExecutorDeserializeTime(v: Long) = _executorDeserializeTime.setValue(v) - private[spark] def setExecutorRunTime(v: Long) = _executorRunTime.setValue(v) - private[spark] def setResultSize(v: Long) = _resultSize.setValue(v) - private[spark] def setJvmGCTime(v: Long) = _jvmGCTime.setValue(v) - private[spark] def setResultSerializationTime(v: Long) = _resultSerializationTime.setValue(v) - private[spark] def incMemoryBytesSpilled(v: Long) = _memoryBytesSpilled.add(v) - private[spark] def incDiskBytesSpilled(v: Long) = _diskBytesSpilled.add(v) - private[spark] def incPeakExecutionMemory(v: Long) = _peakExecutionMemory.add(v) - private[spark] def incUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]) = + private[spark] def setExecutorDeserializeTime(v: Long): Unit = + _executorDeserializeTime.setValue(v) + private[spark] def setExecutorRunTime(v: Long): Unit = _executorRunTime.setValue(v) + private[spark] def setResultSize(v: Long): Unit = _resultSize.setValue(v) + private[spark] def setJvmGCTime(v: Long): Unit = _jvmGCTime.setValue(v) + private[spark] def setResultSerializationTime(v: Long): Unit = + _resultSerializationTime.setValue(v) + private[spark] def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) + private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) + private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + private[spark] def incUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.add(v) - private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]) = + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 0cd0dc487a28..c5b0ad294a7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -56,6 +56,8 @@ class PartitionBatchPruningSuite override protected def beforeEach(): Unit = { super.beforeEach() + // This creates accumulators, which get cleaned up after every single test, + // so we need to do this before every test. val pruningData = sparkContext.makeRDD((1 to 100).map { key => val string = if (((key - 1) / 10) % 2 == 0) null else key.toString TestData(key, string) From 5ca949f23170ca8b1763800ca106024ff29c85cf Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 17:55:44 -0800 Subject: [PATCH 57/65] Fix style --- core/src/main/scala/org/apache/spark/AccumulableParam.scala | 2 +- core/src/main/scala/org/apache/spark/InternalAccumulator.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/AccumulableParam.scala b/core/src/main/scala/org/apache/spark/AccumulableParam.scala index 7c6defe4166a..0a97816d2856 100644 --- a/core/src/main/scala/org/apache/spark/AccumulableParam.scala +++ b/core/src/main/scala/org/apache/spark/AccumulableParam.scala @@ -23,7 +23,7 @@ import scala.collection.generic.Growable import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.storage.{BlockStatus, BlockId} +import org.apache.spark.storage.{BlockId, BlockStatus} /** diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 7904e081e4fd..3dc7835d4653 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.storage.{BlockStatus, BlockId} +import org.apache.spark.storage.{BlockId, BlockStatus} /** From 109fec3c494d9bbaf773ad2790991e0f65c6bfb7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 18:34:48 -0800 Subject: [PATCH 58/65] Add InternalAccumulatorsSuite --- .../apache/spark/InternalAccumulator.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 172 ++--------- .../spark/InternalAccumulatorSuite.scala | 278 ++++++++++++++++++ 3 files changed, 308 insertions(+), 144 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 3dc7835d4653..cb202a21b90a 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -173,7 +173,7 @@ private[spark] object InternalAccumulator { /** * Accumulators for tracking output metrics. */ - private def createOutputAccums(): Seq[Accumulator[_]] = { + def createOutputAccums(): Seq[Accumulator[_]] = { Seq[String]( output.WRITE_METHOD, output.BYTES_WRITTEN, diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index a0c86d76bcaf..2144032cee79 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.scheduler._ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext { - import InternalAccumulator._ + import AccumulatorParam._ implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = new AccumulableParam[mutable.Set[A], A] { @@ -159,148 +159,34 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(!Accumulators.originals.get(accId).isDefined) } - test("internal accumulators in TaskContext") { - sc = new SparkContext("local", "test") - val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null) - val accumUpdates = taskContext.taskMetrics.accumulatorUpdates() - assert(accumUpdates.size > 0) - assert(accumUpdates.forall(_.internal)) - val testAccum = taskContext.taskMetrics.getAccum(TEST_ACCUM) - assert(accumUpdates.exists(_.id == testAccum.id)) - } - - test("internal accumulators in a stage") { - val listener = new SaveInfoListener - val numPartitions = 10 - sc = new SparkContext("local", "test") - sc.addSparkListener(listener) - // Have each task add 1 to the internal accumulator - val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - val stageInfos = listener.getCompletedStageInfos - val taskInfos = listener.getCompletedTaskInfos - assert(stageInfos.size === 1) - assert(taskInfos.size === numPartitions) - // The accumulator values should be merged in the stage - val stageAccum = findTestAccum(stageInfos.head.accumulables.values) - assert(stageAccum.value.get.toString.toLong === numPartitions) - // The accumulator should be updated locally on each task - val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findTestAccum(taskInfo.accumulables) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toString.toLong === 1L) - taskAccum.value.get.toString.toLong - } - // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions - assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) - } - rdd.count() - } - - test("internal accumulators in multiple stages") { - val listener = new SaveInfoListener - val numPartitions = 10 - sc = new SparkContext("local", "test") - sc.addSparkListener(listener) - // Each stage creates its own set of internal accumulators so the - // values for the same metric should not be mixed up across stages - val rdd = sc.parallelize(1 to 100, numPartitions) - .map { i => (i, i) } - .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 - iter - } - .reduceByKey { case (x, y) => x + y } - .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 10 - iter - } - .repartition(numPartitions * 2) - .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 100 - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - // We ran 3 stages, and the accumulator values should be distinct - val stageInfos = listener.getCompletedStageInfos - assert(stageInfos.size === 3) - val (firstStageAccum, secondStageAccum, thirdStageAccum) = - (findTestAccum(stageInfos(0).accumulables.values), - findTestAccum(stageInfos(1).accumulables.values), - findTestAccum(stageInfos(2).accumulables.values)) - assert(firstStageAccum.value.get.toString.toLong === numPartitions) - assert(secondStageAccum.value.get.toString.toLong === numPartitions * 10) - assert(thirdStageAccum.value.get.toString.toLong === numPartitions * 2 * 100) - } - rdd.count() - } - - test("internal accumulators in fully resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => true) // fail all tasks - } - - test("internal accumulators in partially resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => i % 2 == 0) // fail a subset - } - - /** - * Return the accumulable info that matches the specified name. - */ - private def findTestAccum(accums: Iterable[AccumulableInfo]): AccumulableInfo = { - accums.find { a => a.name == TEST_ACCUM }.getOrElse { - fail(s"unable to find internal accumulator called $TEST_ACCUM") - } - } - - /** - * Test whether internal accumulators are merged properly if some tasks fail. - */ - private def testInternalAccumulatorsWithFailedTasks(failCondition: (Int => Boolean)): Unit = { - val listener = new SaveInfoListener - val numPartitions = 10 - val numFailedPartitions = (0 until numPartitions).count(failCondition) - // This says use 1 core and retry tasks up to 2 times - sc = new SparkContext("local[1, 2]", "test") - sc.addSparkListener(listener) - val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => - val taskContext = TaskContext.get() - taskContext.taskMetrics.getAccum(TEST_ACCUM) += 1 - // Fail the first attempts of a subset of the tasks - if (failCondition(i) && taskContext.attemptNumber() == 0) { - throw new Exception("Failing a task intentionally.") - } - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - val stageInfos = listener.getCompletedStageInfos - val taskInfos = listener.getCompletedTaskInfos - assert(stageInfos.size === 1) - assert(taskInfos.size === numPartitions + numFailedPartitions) - val stageAccum = findTestAccum(stageInfos.head.accumulables.values) - // We should not double count values in the merged accumulator - assert(stageAccum.value.get.toString.toLong === numPartitions) - val taskAccumValues = taskInfos.flatMap { taskInfo => - if (!taskInfo.failed) { - // If a task succeeded, its update value should always be 1 - val taskAccum = findTestAccum(taskInfo.accumulables) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toString.toLong === 1L) - Some(taskAccum.value.get.toString.toLong) - } else { - // If a task failed, we should not get its accumulator values - assert(taskInfo.accumulables.isEmpty) - None - } - } - assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) - } - rdd.count() + test("string accumulator param") { + val acc = new Accumulator("", StringAccumulatorParam, Some("darkness")) + assert(acc.value === "") + acc.setValue("feeds") + assert(acc.value === "feeds") + acc.add("your") + assert(acc.value === "your") // value is overwritten, not concatenated + acc += "soul" + assert(acc.value === "soul") + acc ++= "with" + assert(acc.value === "with") + acc.merge("kindness") + assert(acc.value === "kindness") + } + + test("list accumulator param") { + val acc = new Accumulator(Seq.empty[Int], new ListAccumulatorParam[Int], Some("numbers")) + assert(acc.value === Seq.empty[Int]) + acc.add(Seq(1, 2)) + assert(acc.value === Seq(1, 2)) + acc += Seq(3, 4) + assert(acc.value === Seq(1, 2, 3, 4)) + acc ++= Seq(5, 6) + assert(acc.value === Seq(1, 2, 3, 4, 5, 6)) + acc.merge(Seq(7, 8)) + assert(acc.value === Seq(1, 2, 3, 4, 5, 6, 7, 8)) + acc.setValue(Seq(9, 10)) + assert(acc.value === Seq(9, 10)) } } diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala new file mode 100644 index 000000000000..2a72f2f7d9c3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + + +import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.scheduler.AccumulableInfo + + +class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { + import InternalAccumulator._ + import AccumulatorParam._ + + test("get param") { + assert(getParam(EXECUTOR_DESERIALIZE_TIME) === LongAccumulatorParam) + assert(getParam(EXECUTOR_RUN_TIME) === LongAccumulatorParam) + assert(getParam(RESULT_SIZE) === LongAccumulatorParam) + assert(getParam(JVM_GC_TIME) === LongAccumulatorParam) + assert(getParam(RESULT_SERIALIZATION_TIME) === LongAccumulatorParam) + assert(getParam(MEMORY_BYTES_SPILLED) === LongAccumulatorParam) + assert(getParam(DISK_BYTES_SPILLED) === LongAccumulatorParam) + assert(getParam(PEAK_EXECUTION_MEMORY) === LongAccumulatorParam) + assert(getParam(UPDATED_BLOCK_STATUSES) === UpdatedBlockStatusesAccumulatorParam) + assert(getParam(TEST_ACCUM) === LongAccumulatorParam) + // shuffle read + assert(getParam(shuffleRead.REMOTE_BLOCKS_FETCHED) === IntAccumulatorParam) + assert(getParam(shuffleRead.LOCAL_BLOCKS_FETCHED) === IntAccumulatorParam) + assert(getParam(shuffleRead.REMOTE_BYTES_READ) === LongAccumulatorParam) + assert(getParam(shuffleRead.LOCAL_BYTES_READ) === LongAccumulatorParam) + assert(getParam(shuffleRead.FETCH_WAIT_TIME) === LongAccumulatorParam) + assert(getParam(shuffleRead.RECORDS_READ) === LongAccumulatorParam) + // shuffle write + assert(getParam(shuffleWrite.BYTES_WRITTEN) === LongAccumulatorParam) + assert(getParam(shuffleWrite.RECORDS_WRITTEN) === LongAccumulatorParam) + assert(getParam(shuffleWrite.WRITE_TIME) === LongAccumulatorParam) + // input + assert(getParam(input.READ_METHOD) === StringAccumulatorParam) + assert(getParam(input.RECORDS_READ) === LongAccumulatorParam) + assert(getParam(input.BYTES_READ) === LongAccumulatorParam) + // output + assert(getParam(output.WRITE_METHOD) === StringAccumulatorParam) + assert(getParam(output.RECORDS_WRITTEN) === LongAccumulatorParam) + assert(getParam(output.BYTES_WRITTEN) === LongAccumulatorParam) + intercept[AssertionError] { + getParam("something that does not start with the right prefix") + } + } + + test("create by name") { + val executorRunTime = create(EXECUTOR_RUN_TIME) + val updatedBlockStatuses = create(UPDATED_BLOCK_STATUSES) + val shuffleRemoteBlocksRead = create(shuffleRead.REMOTE_BLOCKS_FETCHED) + val inputReadMethod = create(input.READ_METHOD) + assert(executorRunTime.isInstanceOf[Accumulator[Long]]) + assert(executorRunTime.name === Some(EXECUTOR_RUN_TIME)) + assert(updatedBlockStatuses.isInstanceOf[Accumulator[Seq[(BlockId, BlockStatus)]]]) + assert(updatedBlockStatuses.name === Some(UPDATED_BLOCK_STATUSES)) + assert(shuffleRemoteBlocksRead.isInstanceOf[Accumulator[Int]]) + assert(shuffleRemoteBlocksRead.name === Some(shuffleRead.REMOTE_BLOCKS_FETCHED)) + assert(inputReadMethod.isInstanceOf[Accumulator[String]]) + assert(inputReadMethod.name === Some(input.READ_METHOD)) + } + + test("create") { + val accums = create() + val shuffleReadAccums = createShuffleReadAccums() + val shuffleWriteAccums = createShuffleWriteAccums() + val inputAccums = createInputAccums() + val outputAccums = createOutputAccums() + // assert they're all internal + assert(accums.forall(_.isInternal)) + assert(shuffleReadAccums.forall(_.isInternal)) + assert(shuffleWriteAccums.forall(_.isInternal)) + assert(inputAccums.forall(_.isInternal)) + assert(outputAccums.forall(_.isInternal)) + // assert they all count on failures + assert(accums.forall(_.countFailedValues)) + assert(shuffleReadAccums.forall(_.countFailedValues)) + assert(shuffleWriteAccums.forall(_.countFailedValues)) + assert(inputAccums.forall(_.countFailedValues)) + assert(outputAccums.forall(_.countFailedValues)) + // assert they all have names + assert(accums.forall(_.name.isDefined)) + assert(shuffleReadAccums.forall(_.name.isDefined)) + assert(shuffleWriteAccums.forall(_.name.isDefined)) + assert(inputAccums.forall(_.name.isDefined)) + assert(outputAccums.forall(_.name.isDefined)) + // assert `accums` is a strict superset of the others + val accumNames = accums.map(_.name.get).toSet + val shuffleReadAccumNames = shuffleReadAccums.map(_.name.get).toSet + val shuffleWriteAccumNames = shuffleWriteAccums.map(_.name.get).toSet + val inputAccumNames = inputAccums.map(_.name.get).toSet + val outputAccumNames = outputAccums.map(_.name.get).toSet + assert(shuffleReadAccumNames.subsetOf(accumNames)) + assert(shuffleWriteAccumNames.subsetOf(accumNames)) + assert(inputAccumNames.subsetOf(accumNames)) + assert(outputAccumNames.subsetOf(accumNames)) + } + + test("naming") { + val accums = create() + val shuffleReadAccums = createShuffleReadAccums() + val shuffleWriteAccums = createShuffleWriteAccums() + val inputAccums = createInputAccums() + val outputAccums = createOutputAccums() + // assert that prefixes are properly namespaced + assert(SHUFFLE_READ_METRICS_PREFIX.startsWith(METRICS_PREFIX)) + assert(SHUFFLE_WRITE_METRICS_PREFIX.startsWith(METRICS_PREFIX)) + assert(INPUT_METRICS_PREFIX.startsWith(METRICS_PREFIX)) + assert(OUTPUT_METRICS_PREFIX.startsWith(METRICS_PREFIX)) + assert(accums.forall(_.name.get.startsWith(METRICS_PREFIX))) + // assert they all start with the expected prefixes + assert(shuffleReadAccums.forall(_.name.get.startsWith(SHUFFLE_READ_METRICS_PREFIX))) + assert(shuffleWriteAccums.forall(_.name.get.startsWith(SHUFFLE_WRITE_METRICS_PREFIX))) + assert(inputAccums.forall(_.name.get.startsWith(INPUT_METRICS_PREFIX))) + assert(outputAccums.forall(_.name.get.startsWith(OUTPUT_METRICS_PREFIX))) + } + + test("internal accumulators in TaskContext") { + sc = new SparkContext("local", "test") + val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null) + val accumUpdates = taskContext.taskMetrics.accumulatorUpdates() + assert(accumUpdates.size > 0) + assert(accumUpdates.forall(_.internal)) + val testAccum = taskContext.taskMetrics.getAccum(TEST_ACCUM) + assert(accumUpdates.exists(_.id == testAccum.id)) + } + + test("internal accumulators in a stage") { + val listener = new SaveInfoListener + val numPartitions = 10 + sc = new SparkContext("local", "test") + sc.addSparkListener(listener) + // Have each task add 1 to the internal accumulator + val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 + iter + } + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { _ => + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findTestAccum(stageInfos.head.accumulables.values) + assert(stageAccum.value.get.toString.toLong === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo => + val taskAccum = findTestAccum(taskInfo.accumulables) + assert(taskAccum.update.isDefined) + assert(taskAccum.update.get.toString.toLong === 1L) + taskAccum.value.get.toString.toLong + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) + } + rdd.count() + } + + test("internal accumulators in multiple stages") { + val listener = new SaveInfoListener + val numPartitions = 10 + sc = new SparkContext("local", "test") + sc.addSparkListener(listener) + // Each stage creates its own set of internal accumulators so the + // values for the same metric should not be mixed up across stages + val rdd = sc.parallelize(1 to 100, numPartitions) + .map { i => (i, i) } + .mapPartitions { iter => + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 + iter + } + .reduceByKey { case (x, y) => x + y } + .mapPartitions { iter => + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 10 + iter + } + .repartition(numPartitions * 2) + .mapPartitions { iter => + TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 100 + iter + } + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { _ => + // We ran 3 stages, and the accumulator values should be distinct + val stageInfos = listener.getCompletedStageInfos + assert(stageInfos.size === 3) + val (firstStageAccum, secondStageAccum, thirdStageAccum) = + (findTestAccum(stageInfos(0).accumulables.values), + findTestAccum(stageInfos(1).accumulables.values), + findTestAccum(stageInfos(2).accumulables.values)) + assert(firstStageAccum.value.get.toString.toLong === numPartitions) + assert(secondStageAccum.value.get.toString.toLong === numPartitions * 10) + assert(thirdStageAccum.value.get.toString.toLong === numPartitions * 2 * 100) + } + rdd.count() + } + + test("internal accumulators in fully resubmitted stages") { + testInternalAccumulatorsWithFailedTasks((i: Int) => true) // fail all tasks + } + + test("internal accumulators in partially resubmitted stages") { + testInternalAccumulatorsWithFailedTasks((i: Int) => i % 2 == 0) // fail a subset + } + + /** + * Return the accumulable info that matches the specified name. + */ + private def findTestAccum(accums: Iterable[AccumulableInfo]): AccumulableInfo = { + accums.find { a => a.name == TEST_ACCUM }.getOrElse { + fail(s"unable to find internal accumulator called $TEST_ACCUM") + } + } + + /** + * Test whether internal accumulators are merged properly if some tasks fail. + */ + private def testInternalAccumulatorsWithFailedTasks(failCondition: (Int => Boolean)): Unit = { + val listener = new SaveInfoListener + val numPartitions = 10 + val numFailedPartitions = (0 until numPartitions).count(failCondition) + // This says use 1 core and retry tasks up to 2 times + sc = new SparkContext("local[1, 2]", "test") + sc.addSparkListener(listener) + val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => + val taskContext = TaskContext.get() + taskContext.taskMetrics.getAccum(TEST_ACCUM) += 1 + // Fail the first attempts of a subset of the tasks + if (failCondition(i) && taskContext.attemptNumber() == 0) { + throw new Exception("Failing a task intentionally.") + } + iter + } + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { _ => + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions + numFailedPartitions) + val stageAccum = findTestAccum(stageInfos.head.accumulables.values) + // We should not double count values in the merged accumulator + assert(stageAccum.value.get.toString.toLong === numPartitions) + val taskAccumValues = taskInfos.flatMap { taskInfo => + if (!taskInfo.failed) { + // If a task succeeded, its update value should always be 1 + val taskAccum = findTestAccum(taskInfo.accumulables) + assert(taskAccum.update.isDefined) + assert(taskAccum.update.get.toString.toLong === 1L) + Some(taskAccum.value.get.toString.toLong) + } else { + // If a task failed, we should not get its accumulator values + assert(taskInfo.accumulables.isEmpty) + None + } + } + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) + } + rdd.count() + } + +} From a393ade79b3b64623810c4a8c3327f2054041d70 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 14 Jan 2016 23:31:25 -0800 Subject: [PATCH 59/65] Fix JsonProtocolSuite --- .../apache/spark/util/JsonProtocolSuite.scala | 206 +++++++++++------- 1 file changed, 129 insertions(+), 77 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 965e413f4c8c..461eb470a73c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -832,14 +832,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -889,14 +891,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | } @@ -927,21 +931,24 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 2, | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 3, | "Name": "Accumulable3", | "Update": "delta3", | "Value": "val3", - | "Internal": true + | "Internal": true, + | "Count Failed Values": false | } | ] | } @@ -970,21 +977,24 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 2, | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 3, | "Name": "Accumulable3", | "Update": "delta3", | "Value": "val3", - | "Internal": true + | "Internal": true, + | "Count Failed Values": false | } | ] | } @@ -1019,21 +1029,24 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 2, | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 3, | "Name": "Accumulable3", | "Update": "delta3", | "Value": "val3", - | "Internal": true + | "Internal": true, + | "Count Failed Values": false | } | ] | }, @@ -1107,21 +1120,24 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 2, | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 3, | "Name": "Accumulable3", | "Update": "delta3", | "Value": "val3", - | "Internal": true + | "Internal": true, + | "Count Failed Values": false | } | ] | }, @@ -1192,21 +1208,24 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 2, | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 3, | "Name": "Accumulable3", | "Update": "delta3", | "Value": "val3", - | "Internal": true + | "Internal": true, + | "Count Failed Values": false | } | ] | }, @@ -1289,14 +1308,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1351,14 +1372,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1431,14 +1454,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | }, @@ -1529,14 +1554,16 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Name": "Accumulable2", | "Update": "delta2", | "Value": "val2", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | }, | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", - | "Internal": false + | "Internal": false, + | "Count Failed Values": false | } | ] | } @@ -1699,55 +1726,63 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | "Stage Attempt ID": 3, | "Accumulator Updates": [ | { - | "ID": 82, + | "ID": 81, | "Name": "$EXECUTOR_DESERIALIZE_TIME", | "Update": 300, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 83, + | "ID": 82, | "Name": "$EXECUTOR_RUN_TIME", | "Update": 400, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 84, + | "ID": 83, | "Name": "$RESULT_SIZE", | "Update": 500, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 85, + | "ID": 84, | "Name": "$JVM_GC_TIME", | "Update": 600, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 86, + | "ID": 85, | "Name": "$RESULT_SERIALIZATION_TIME", | "Update": 700, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 87, + | "ID": 86, | "Name": "$MEMORY_BYTES_SPILLED", | "Update": 800, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 88, + | "ID": 87, | "Name": "$DISK_BYTES_SPILLED", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 89, + | "ID": 88, | "Name": "$PEAK_EXECUTION_MEMORY", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 90, + | "ID": 89, | "Name": "$UPDATED_BLOCK_STATUSES", | "Update": [ | { @@ -1766,103 +1801,120 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { | } | } | ], - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 91, + | "ID": 90, | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 92, + | "ID": 91, | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 93, + | "ID": 92, | "Name": "${shuffleRead.REMOTE_BYTES_READ}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 94, + | "ID": 93, | "Name": "${shuffleRead.LOCAL_BYTES_READ}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 95, + | "ID": 94, | "Name": "${shuffleRead.FETCH_WAIT_TIME}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 96, + | "ID": 95, | "Name": "${shuffleRead.RECORDS_READ}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 97, + | "ID": 96, | "Name": "${shuffleWrite.BYTES_WRITTEN}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 98, + | "ID": 97, | "Name": "${shuffleWrite.RECORDS_WRITTEN}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 99, + | "ID": 98, | "Name": "${shuffleWrite.WRITE_TIME}", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 100, + | "ID": 99, | "Name": "${input.READ_METHOD}", | "Update": "Hadoop", - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 101, + | "ID": 100, | "Name": "${input.BYTES_READ}", | "Update": 2100, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 102, + | "ID": 101, | "Name": "${input.RECORDS_READ}", | "Update": 21, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 103, + | "ID": 102, | "Name": "${output.WRITE_METHOD}", | "Update": "Hadoop", - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 104, + | "ID": 103, | "Name": "${output.BYTES_WRITTEN}", | "Update": 1200, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 105, + | "ID": 104, | "Name": "${output.RECORDS_WRITTEN}", | "Update": 12, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | }, | { - | "ID": 106, + | "ID": 105, | "Name": "$TEST_ACCUM", | "Update": 0, - | "Internal": true + | "Internal": true, + | "Count Failed Values": true | } | ] | } From e59f3686859084096e7d10e89ed5ce9a9cf3136d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Jan 2016 11:23:08 -0800 Subject: [PATCH 60/65] Add more tests to AccumulatorSuite --- .../org/apache/spark/AccumulatorSuite.scala | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 2144032cee79..a818b93545d2 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -159,6 +159,77 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(!Accumulators.originals.get(accId).isDefined) } + test("get accum") { + sc = new SparkContext("local", "test") + // Don't register with SparkContext for cleanup + var acc = new Accumulable[Int, Int](0, IntAccumulatorParam, None, true, true) + val accId = acc.id + val ref = WeakReference(acc) + assert(ref.get.isDefined) + Accumulators.register(ref.get.get) + + // Remove the explicit reference to it and allow weak reference to get garbage collected + acc = null + System.gc() + assert(ref.get.isEmpty) + + // Getting a garbage collected accum should throw error + intercept[IllegalAccessError] { + Accumulators.get(accId) + } + + // 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. + val acc2 = new Accumulable[Long, Long](0L, LongAccumulatorParam, None, true, true) + Accumulators.register(acc2) + assert(Accumulators.get(acc2.id) === Some(acc2)) + + // Getting an accumulator that does not exist should return None + assert(Accumulators.get(100000).isEmpty) + } + + test("only external accums are automatically registered") { + val accEx = new Accumulator(0, IntAccumulatorParam, Some("external"), internal = false) + val accIn = new Accumulator(0, IntAccumulatorParam, Some("internal"), internal = true) + assert(!accEx.isInternal) + assert(accIn.isInternal) + assert(Accumulators.get(accEx.id).isDefined) + assert(Accumulators.get(accIn.id).isEmpty) + } + + test("copy") { + val acc1 = new Accumulable[Long, Long](456L, LongAccumulatorParam, Some("x"), true, false) + val acc2 = acc1.copy() + assert(acc1.id === acc2.id) + assert(acc1.value === acc2.value) + assert(acc1.name === acc2.name) + assert(acc1.isInternal === acc2.isInternal) + assert(acc1.countFailedValues === acc2.countFailedValues) + assert(acc1 !== acc2) + // Modifying one does not affect the other + acc1.add(44L) + assert(acc1.value === 500L) + assert(acc2.value === 456L) + acc2.add(144L) + assert(acc1.value === 500L) + assert(acc2.value === 600L) + } + + test("register multiple accums with same ID") { + // Make sure these are internal accums so we don't automatically register them already + val acc1 = new Accumulable[Int, Int](0, IntAccumulatorParam, None, true, true) + val acc2 = acc1.copy() + assert(acc1 !== acc2) + assert(acc1.id === acc2.id) + assert(Accumulators.originals.isEmpty) + assert(Accumulators.get(acc1.id).isEmpty) + Accumulators.register(acc1) + Accumulators.register(acc2) + // The second one does not override the first one + assert(Accumulators.originals.size === 1) + assert(Accumulators.get(acc1.id) === Some(acc1)) + } + test("string accumulator param") { val acc = new Accumulator("", StringAccumulatorParam, Some("darkness")) assert(acc.value === "") From b17e848fb0b860ba72011b9df7793dcd2f78d293 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Jan 2016 11:43:22 -0800 Subject: [PATCH 61/65] Fix style --- .../test/scala/org/apache/spark/InternalAccumulatorSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 2a72f2f7d9c3..e487edc510d7 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark - -import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.storage.{BlockId, BlockStatus} class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { From 06b958e1542b6e4760df64394cd52ee73e97ad82 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Jan 2016 12:32:02 -0800 Subject: [PATCH 62/65] Add test for accum updates during failures --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/InternalAccumulatorSuite.scala | 8 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 60 +++++++++++++++++-- 3 files changed, 59 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cf696f80f818..e1ab434e475e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1301,7 +1301,7 @@ class DAGScheduler( // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits case exceptionFailure: ExceptionFailure => - // Tasks failed with exceptions might still have accumulator updates. TODO: write a test. + // Tasks failed with exceptions might still have accumulator updates. updateAccumulators(event) case TaskResultLost => diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index e487edc510d7..395ffed2b1d9 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -65,14 +65,14 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { val updatedBlockStatuses = create(UPDATED_BLOCK_STATUSES) val shuffleRemoteBlocksRead = create(shuffleRead.REMOTE_BLOCKS_FETCHED) val inputReadMethod = create(input.READ_METHOD) - assert(executorRunTime.isInstanceOf[Accumulator[Long]]) assert(executorRunTime.name === Some(EXECUTOR_RUN_TIME)) - assert(updatedBlockStatuses.isInstanceOf[Accumulator[Seq[(BlockId, BlockStatus)]]]) assert(updatedBlockStatuses.name === Some(UPDATED_BLOCK_STATUSES)) - assert(shuffleRemoteBlocksRead.isInstanceOf[Accumulator[Int]]) assert(shuffleRemoteBlocksRead.name === Some(shuffleRead.REMOTE_BLOCKS_FETCHED)) - assert(inputReadMethod.isInstanceOf[Accumulator[String]]) assert(inputReadMethod.name === Some(input.READ_METHOD)) + assert(executorRunTime.value.isInstanceOf[Long]) + assert(updatedBlockStatuses.value.isInstanceOf[Seq[(BlockId, BlockStatus)]]) + assert(shuffleRemoteBlocksRead.value.isInstanceOf[Int]) + assert(inputReadMethod.value.isInstanceOf[String]) } test("create") { 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 b5532c0e2d8f..761874d4aeb6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -191,7 +190,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def beforeEach(): Unit = { super.beforeEach() - sc = new SparkContext("local", "DAGSchedulerSuite") + // This means use 1 core and allow up to 4 failed tasks + sc = new SparkContext("local[1, 4]", "DAGSchedulerSuite") sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() sparkListener.failedStages.clear() @@ -1574,6 +1574,48 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assertDataStructuresEmpty() } + test("accumulators are updated on exception failures") { + val acc1 = sc.accumulator(0L, "ingenieur") + val acc2 = sc.accumulator(0L, "boulangere") + val acc3 = sc.accumulator(0L, "agriculteur") + assert(Accumulators.get(acc1.id).isDefined) + assert(Accumulators.get(acc2.id).isDefined) + assert(Accumulators.get(acc3.id).isDefined) + val accInfo1 = new AccumulableInfo(acc1.id, acc1.name.get, Some(15L), None) + val accInfo2 = new AccumulableInfo(acc2.id, acc2.name.get, Some(13L), None) + val accInfo3 = new AccumulableInfo(acc3.id, acc3.name.get, Some(18L), None) + val accumUpdates = Seq(accInfo1, accInfo2, accInfo3) + val exceptionFailure = new ExceptionFailure(new SparkException("fondue?"), accumUpdates) + submit(new MyRDD(sc, 1, Nil), Array(0)) + runEvent(makeCompletionEvent(taskSets.head.tasks.head, exceptionFailure, "result")) + assert(Accumulators.get(acc1.id).get.value === 15L) + assert(Accumulators.get(acc2.id).get.value === 13L) + assert(Accumulators.get(acc3.id).get.value === 18L) + } + + test("accumulators are updated on exception failures (end-to-end)") { + import AccumulatorParam._ + // Create 2 accumulators, one that counts failed values and another that doesn't + val acc1 = new Accumulator( + 0L, LongAccumulatorParam, Some("ingenieur"), internal = false, countFailedValues = true) + val acc2 = new Accumulator( + 0L, LongAccumulatorParam, Some("boulangere"), internal = false, countFailedValues = false) + // Fail first 3 attempts of every task. This means each task should be run 4 times. + sc.parallelize(1 to 10, 10).map { i => + acc1 += 1 + acc2 += 1 + if (TaskContext.get.attemptNumber() <= 2) { + throw new Exception("you did something wrong") + } else { + 0 + } + }.count() + // The one that counts failed values should be 4x the one that didn't, + // since we ran each task 4 times + assert(Accumulators.get(acc1.id).get.value === 40L) + assert(Accumulators.get(acc2.id).get.value === 10L) + } + test("reduce tasks should be placed locally with map output") { // Create an shuffleMapRdd with 1 partition val shuffleMapRdd = new MyRDD(sc, 1, Nil) @@ -1931,10 +1973,16 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou result: Any, extraAccumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { - val accumUpdates = task.initialAccumulators.map { a => - new AccumulableInfo(a.id, a.name.get, Some(a.zero), None, a.isInternal, a.countFailedValues) - } ++ extraAccumUpdates - CompletionEvent(task, reason, result, accumUpdates, taskInfo) + val accumUpdates = reason match { + case Success => + task.initialAccumulators.map { a => + new AccumulableInfo( + a.id, a.name.get, Some(a.zero), None, a.isInternal, a.countFailedValues) + } + case ef: ExceptionFailure => ef.accumUpdates + case _ => Seq.empty[AccumulableInfo] + } + CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, taskInfo) } } From 28346e51fc51e78abe17a2c53d90f6754c4eddcb Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Jan 2016 15:10:38 -0800 Subject: [PATCH 63/65] Add tests for TaskMetrics --- .../apache/spark/executor/InputMetrics.scala | 2 +- .../spark/executor/ShuffleWriteMetrics.scala | 6 +- .../apache/spark/executor/TaskMetrics.scala | 4 +- .../status/api/v1/AllStagesResource.scala | 4 +- .../org/apache/spark/ui/jobs/StagePage.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../spark/executor/TaskMetricsSuite.scala | 463 +++++++++++++++++- .../storage/DiskBlockObjectWriterSuite.scala | 8 +- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- 9 files changed, 473 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 060ffdd15246..fb2f3bb99031 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -79,7 +79,7 @@ class InputMetrics private ( */ def readMethod: DataReadMethod.Value = DataReadMethod.withName(_readMethod.localValue) - private[spark] def setBytesRead(v: Long): Unit = _bytesRead.add(v) + private[spark] def setBytesRead(v: Long): Unit = _bytesRead.setValue(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = _readMethod.setValue(v.toString) diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index a54961271ab8..efe83854c3d3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -29,7 +29,7 @@ import org.apache.spark.annotation.DeveloperApi class ShuffleWriteMetrics private ( _bytesWritten: Accumulator[Long], _recordsWritten: Accumulator[Long], - _shuffleWriteTime: Accumulator[Long]) + _writeTime: Accumulator[Long]) extends Serializable { private[executor] def this(accumMap: Map[String, Accumulator[_]]) { @@ -65,11 +65,11 @@ class ShuffleWriteMetrics private ( /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. */ - def shuffleWriteTime: Long = _shuffleWriteTime.localValue + def writeTime: Long = _writeTime.localValue private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) - private[spark] def incWriteTime(v: Long): Unit = _shuffleWriteTime.add(v) + private[spark] def incWriteTime(v: Long): Unit = _writeTime.add(v) private[spark] def decBytesWritten(v: Long): Unit = { _bytesWritten.setValue(bytesWritten - v) } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 36b5337a0b82..87adee909d75 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -373,8 +373,8 @@ private[spark] object TaskMetrics extends Logging { * need the latter to post task end events to listeners, so we need to reconstruct the metrics * on the driver. * - * Note: If the task failed, we may return null after attempting to reconstruct the - * [[TaskMetrics]] in vain. + * This assumes the provided updates contain the initial set of accumulators representing + * internal task level metrics. */ def fromAccumulatorUpdates(taskId: Long, accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { // Initial accumulators are passed into the TaskMetrics constructor first because these diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 87996429334d..23b1f7e06a5f 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -216,7 +216,7 @@ private[v1] object AllStagesResource { def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( writeBytes = submetricQuantiles(_.bytesWritten), writeRecords = submetricQuantiles(_.recordsWritten), - writeTime = submetricQuantiles(_.shuffleWriteTime) + writeTime = submetricQuantiles(_.writeTime) ) }.metricOption @@ -288,7 +288,7 @@ private[v1] object AllStagesResource { def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { new ShuffleWriteMetrics( bytesWritten = internal.bytesWritten, - writeTime = internal.shuffleWriteTime, + writeTime = internal.writeTime, recordsWritten = internal.recordsWritten ) } 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 1616132b9895..935382cd68ca 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 @@ -615,7 +615,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val shuffleReadTimeProportion = toProportion(shuffleReadTime) val shuffleWriteTime = (metricsOpt.flatMap(_.shuffleWriteMetrics - .map(_.shuffleWriteTime)).getOrElse(0L) / 1e6).toLong + .map(_.writeTime)).getOrElse(0L) / 1e6).toLong val shuffleWriteTimeProportion = toProportion(shuffleWriteTime) val serializationTime = metricsOpt.map(_.resultSerializationTime).getOrElse(0L) @@ -927,7 +927,7 @@ private[ui] class TaskDataSource( val shuffleWriteRecords = maybeShuffleWrite .map(_.recordsWritten.toString).getOrElse("") - val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.writeTime) val writeTimeSortable = maybeWriteTime.getOrElse(0L) val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => if (ms == 0) "" else UIUtils.formatDuration(ms) 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 99b0efb28aeb..f3c5e33e9c7a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -336,7 +336,7 @@ private[spark] object JsonProtocol { val shuffleWriteMetrics: JValue = taskMetrics.shuffleWriteMetrics.map { wm => ("Shuffle Bytes Written" -> wm.bytesWritten) ~ - ("Shuffle Write Time" -> wm.shuffleWriteTime) ~ + ("Shuffle Write Time" -> wm.writeTime) ~ ("Shuffle Records Written" -> wm.recordsWritten) }.getOrElse(JNothing) val inputMetrics: JValue = diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index e5ec2aa1be35..aea38c946458 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -17,12 +17,463 @@ package org.apache.spark.executor -import org.apache.spark.SparkFunSuite +import org.apache.spark._ +import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel, TestBlockId} -class TaskMetricsSuite extends SparkFunSuite { - test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") { - val taskMetrics = new TaskMetrics() - taskMetrics.mergeShuffleReadMetrics() - assert(taskMetrics.shuffleReadMetrics.isEmpty) + +class TaskMetricsSuite extends SparkFunSuite { self => + import AccumulatorParam._ + import InternalAccumulator._ + import StorageLevel._ + + test("create") { + val internalAccums = InternalAccumulator.create() + val tm1 = new TaskMetrics + val tm2 = new TaskMetrics(internalAccums) + assert(tm1.accumulatorUpdates().size === internalAccums.size) + assert(tm2.accumulatorUpdates().size === internalAccums.size) + val unnamedAccum = new Accumulator(0, IntAccumulatorParam, None, internal = true) + val dupNamedAccum = new Accumulator(0, IntAccumulatorParam, Some(RESULT_SIZE), internal = true) + val externalAccum = new Accumulator(0, IntAccumulatorParam, Some("x")) + val internalAccums2 = internalAccums ++ Seq(unnamedAccum) + val internalAccums3 = internalAccums ++ Seq(dupNamedAccum) + val internalAccums4 = internalAccums ++ Seq(externalAccum) + // TaskMetrics constructor expects minimal set of initial accumulators + intercept[AssertionError] { new TaskMetrics(Seq.empty[Accumulator[_]]) } + // initial accums must be named + intercept[AssertionError] { new TaskMetrics(internalAccums2) } + // initial accums must not have duplicate names + intercept[AssertionError] { new TaskMetrics(internalAccums3) } + // initial accums must be internal + intercept[AssertionError] { new TaskMetrics(internalAccums4) } + } + + test("mutating values") { + val accums = InternalAccumulator.create() + val tm = new TaskMetrics(accums) + // initial values + assertValueEquals(tm, _.executorDeserializeTime, accums, EXECUTOR_DESERIALIZE_TIME, 0L) + assertValueEquals(tm, _.executorRunTime, accums, EXECUTOR_RUN_TIME, 0L) + assertValueEquals(tm, _.resultSize, accums, RESULT_SIZE, 0L) + assertValueEquals(tm, _.jvmGCTime, accums, JVM_GC_TIME, 0L) + assertValueEquals(tm, _.resultSerializationTime, accums, RESULT_SERIALIZATION_TIME, 0L) + assertValueEquals(tm, _.memoryBytesSpilled, accums, MEMORY_BYTES_SPILLED, 0L) + assertValueEquals(tm, _.diskBytesSpilled, accums, DISK_BYTES_SPILLED, 0L) + assertValueEquals(tm, _.peakExecutionMemory, accums, PEAK_EXECUTION_MEMORY, 0L) + assertValueEquals(tm, _.updatedBlockStatuses, accums, UPDATED_BLOCK_STATUSES, + Seq.empty[(BlockId, BlockStatus)]) + // set or increment values + tm.setExecutorDeserializeTime(100L) + tm.setExecutorDeserializeTime(1L) // overwrite + tm.setExecutorRunTime(200L) + tm.setExecutorRunTime(2L) + tm.setResultSize(300L) + tm.setResultSize(3L) + tm.setJvmGCTime(400L) + tm.setJvmGCTime(4L) + tm.setResultSerializationTime(500L) + tm.setResultSerializationTime(5L) + tm.incMemoryBytesSpilled(600L) + tm.incMemoryBytesSpilled(6L) // add + tm.incDiskBytesSpilled(700L) + tm.incDiskBytesSpilled(7L) + tm.incPeakExecutionMemory(800L) + tm.incPeakExecutionMemory(8L) + val block1 = (TestBlockId("a"), BlockStatus(MEMORY_ONLY, 1L, 2L)) + val block2 = (TestBlockId("b"), BlockStatus(MEMORY_ONLY, 3L, 4L)) + tm.incUpdatedBlockStatuses(Seq(block1)) + tm.incUpdatedBlockStatuses(Seq(block2)) + // assert new values exist + assertValueEquals(tm, _.executorDeserializeTime, accums, EXECUTOR_DESERIALIZE_TIME, 1L) + assertValueEquals(tm, _.executorRunTime, accums, EXECUTOR_RUN_TIME, 2L) + assertValueEquals(tm, _.resultSize, accums, RESULT_SIZE, 3L) + assertValueEquals(tm, _.jvmGCTime, accums, JVM_GC_TIME, 4L) + assertValueEquals(tm, _.resultSerializationTime, accums, RESULT_SERIALIZATION_TIME, 5L) + assertValueEquals(tm, _.memoryBytesSpilled, accums, MEMORY_BYTES_SPILLED, 606L) + assertValueEquals(tm, _.diskBytesSpilled, accums, DISK_BYTES_SPILLED, 707L) + assertValueEquals(tm, _.peakExecutionMemory, accums, PEAK_EXECUTION_MEMORY, 808L) + assertValueEquals(tm, _.updatedBlockStatuses, accums, UPDATED_BLOCK_STATUSES, + Seq(block1, block2)) + } + + test("mutating shuffle read metrics values") { + import shuffleRead._ + val accums = InternalAccumulator.create() + val tm = new TaskMetrics(accums) + def assertValueEquals[T](tmValue: ShuffleReadMetrics => T, name: String, value: T): Unit = { + self.assertValueEquals(tm, tm => tmValue(tm.shuffleReadMetrics.get), accums, name, value) + } + // create shuffle read metrics + assert(tm.shuffleReadMetrics.isEmpty) + tm.registerTempShuffleReadMetrics() + tm.mergeShuffleReadMetrics() + assert(tm.shuffleReadMetrics.isDefined) + val sr = tm.shuffleReadMetrics.get + // initial values + assertValueEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 0) + assertValueEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 0) + assertValueEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 0L) + assertValueEquals(_.localBytesRead, LOCAL_BYTES_READ, 0L) + assertValueEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 0L) + assertValueEquals(_.recordsRead, RECORDS_READ, 0L) + // set and increment values + sr.setRemoteBlocksFetched(100) + sr.setRemoteBlocksFetched(10) + sr.incRemoteBlocksFetched(1) // 10 + 1 + sr.incRemoteBlocksFetched(1) // 10 + 1 + 1 + sr.setLocalBlocksFetched(200) + sr.setLocalBlocksFetched(20) + sr.incLocalBlocksFetched(2) + sr.incLocalBlocksFetched(2) + sr.setRemoteBytesRead(300L) + sr.setRemoteBytesRead(30L) + sr.incRemoteBytesRead(3L) + sr.incRemoteBytesRead(3L) + sr.setLocalBytesRead(400L) + sr.setLocalBytesRead(40L) + sr.incLocalBytesRead(4L) + sr.incLocalBytesRead(4L) + sr.setFetchWaitTime(500L) + sr.setFetchWaitTime(50L) + sr.incFetchWaitTime(5L) + sr.incFetchWaitTime(5L) + sr.setRecordsRead(600L) + sr.setRecordsRead(60L) + sr.incRecordsRead(6L) + sr.incRecordsRead(6L) + // assert new values exist + assertValueEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 12) + assertValueEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 24) + assertValueEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 36L) + assertValueEquals(_.localBytesRead, LOCAL_BYTES_READ, 48L) + assertValueEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 60L) + assertValueEquals(_.recordsRead, RECORDS_READ, 72L) + } + + test("mutating shuffle write metrics values") { + import shuffleWrite._ + val accums = InternalAccumulator.create() + val tm = new TaskMetrics(accums) + def assertValueEquals[T](tmValue: ShuffleWriteMetrics => T, name: String, value: T): Unit = { + self.assertValueEquals(tm, tm => tmValue(tm.shuffleWriteMetrics.get), accums, name, value) + } + // create shuffle write metrics + assert(tm.shuffleWriteMetrics.isEmpty) + tm.registerShuffleWriteMetrics() + assert(tm.shuffleWriteMetrics.isDefined) + val sw = tm.shuffleWriteMetrics.get + // initial values + assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 0L) + assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assertValueEquals(_.writeTime, WRITE_TIME, 0L) + // increment and decrement values + sw.incBytesWritten(100L) + sw.incBytesWritten(10L) // 100 + 10 + sw.decBytesWritten(1L) // 100 + 10 - 1 + sw.decBytesWritten(1L) // 100 + 10 - 1 - 1 + sw.incRecordsWritten(200L) + sw.incRecordsWritten(20L) + sw.decRecordsWritten(2L) + sw.decRecordsWritten(2L) + sw.incWriteTime(300L) + sw.incWriteTime(30L) + // assert new values exist + assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 108L) + assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 216L) + assertValueEquals(_.writeTime, WRITE_TIME, 330L) + } + + test("mutating input metrics values") { + import input._ + val accums = InternalAccumulator.create() + val tm = new TaskMetrics(accums) + def assertValueEquals(tmValue: InputMetrics => Any, name: String, value: Any): Unit = { + self.assertValueEquals(tm, tm => tmValue(tm.inputMetrics.get), accums, name, value, + (x: Any, y: Any) => assert(x.toString === y.toString)) + } + // create input metrics + assert(tm.inputMetrics.isEmpty) + tm.registerInputMetrics(DataReadMethod.Memory) + assert(tm.inputMetrics.isDefined) + val in = tm.inputMetrics.get + // initial values + assertValueEquals(_.bytesRead, BYTES_READ, 0L) + assertValueEquals(_.recordsRead, RECORDS_READ, 0L) + assertValueEquals(_.readMethod, READ_METHOD, DataReadMethod.Memory) + // set and increment values + in.setBytesRead(1L) + in.setBytesRead(2L) + in.incRecordsRead(1L) + in.incRecordsRead(2L) + in.setReadMethod(DataReadMethod.Disk) + // assert new values exist + assertValueEquals(_.bytesRead, BYTES_READ, 2L) + assertValueEquals(_.recordsRead, RECORDS_READ, 3L) + assertValueEquals(_.readMethod, READ_METHOD, DataReadMethod.Disk) + } + + test("mutating output metrics values") { + import output._ + val accums = InternalAccumulator.create() + val tm = new TaskMetrics(accums) + def assertValueEquals(tmValue: OutputMetrics => Any, name: String, value: Any): Unit = { + self.assertValueEquals(tm, tm => tmValue(tm.outputMetrics.get), accums, name, value, + (x: Any, y: Any) => assert(x.toString === y.toString)) + } + // create input metrics + assert(tm.outputMetrics.isEmpty) + tm.registerOutputMetrics(DataWriteMethod.Hadoop) + assert(tm.outputMetrics.isDefined) + val out = tm.outputMetrics.get + // initial values + assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 0L) + assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assertValueEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) + // set values + out.setBytesWritten(1L) + out.setBytesWritten(2L) + out.setRecordsWritten(3L) + out.setRecordsWritten(4L) + out.setWriteMethod(DataWriteMethod.Hadoop) + // assert new values exist + assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 2L) + assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 4L) + // Note: this doesn't actually test anything, but there's only one DataWriteMethod + // so we can't set it to anything else + assertValueEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) + } + + test("merging multiple shuffle read metrics") { + val tm = new TaskMetrics + assert(tm.shuffleReadMetrics.isEmpty) + val sr1 = tm.registerTempShuffleReadMetrics() + val sr2 = tm.registerTempShuffleReadMetrics() + val sr3 = tm.registerTempShuffleReadMetrics() + assert(tm.shuffleReadMetrics.isEmpty) + sr1.setRecordsRead(10L) + sr2.setRecordsRead(10L) + sr1.setFetchWaitTime(1L) + sr2.setFetchWaitTime(2L) + sr3.setFetchWaitTime(3L) + tm.mergeShuffleReadMetrics() + assert(tm.shuffleReadMetrics.isDefined) + val sr = tm.shuffleReadMetrics.get + assert(sr.remoteBlocksFetched === 0L) + assert(sr.recordsRead === 20L) + assert(sr.fetchWaitTime === 6L) + + // SPARK-5701: calling merge without any shuffle deps does nothing + val tm2 = new TaskMetrics + tm2.mergeShuffleReadMetrics() + assert(tm2.shuffleReadMetrics.isEmpty) + } + + test("register multiple shuffle write metrics") { + val tm = new TaskMetrics + val sw1 = tm.registerShuffleWriteMetrics() + val sw2 = tm.registerShuffleWriteMetrics() + assert(sw1 === sw2) + assert(tm.shuffleWriteMetrics.isDefined) + assert(tm.shuffleWriteMetrics.get === sw1) + } + + test("register multiple input metrics") { + val tm = new TaskMetrics + val im1 = tm.registerInputMetrics(DataReadMethod.Memory) + val im2 = tm.registerInputMetrics(DataReadMethod.Memory) + // input metrics with a different read method than the one already registered are ignored + val im3 = tm.registerInputMetrics(DataReadMethod.Hadoop) + assert(im1 === im2) + assert(im1 !== im3) + assert(tm.inputMetrics.isDefined) + assert(tm.inputMetrics.get === im1) + im2.setBytesRead(50L) + im3.setBytesRead(100L) + assert(tm.inputMetrics.get.bytesRead === 50L) + } + + test("register multiple output metrics") { + val tm = new TaskMetrics + val om1 = tm.registerOutputMetrics(DataWriteMethod.Hadoop) + val om2 = tm.registerOutputMetrics(DataWriteMethod.Hadoop) + assert(om1 === om2) + assert(tm.outputMetrics.isDefined) + assert(tm.outputMetrics.get === om1) + } + + test("additional accumulables") { + val internalAccums = InternalAccumulator.create() + val tm = new TaskMetrics(internalAccums) + assert(tm.accumulatorUpdates().size === internalAccums.size) + val acc1 = new Accumulator(0, IntAccumulatorParam, Some("a")) + val acc2 = new Accumulator(0, IntAccumulatorParam, Some("b")) + val acc3 = new Accumulator(0, IntAccumulatorParam, Some("c")) + val acc4 = new Accumulator(0, IntAccumulatorParam, Some("d"), + internal = true, countFailedValues = true) + tm.registerAccumulator(acc1) + tm.registerAccumulator(acc2) + tm.registerAccumulator(acc3) + tm.registerAccumulator(acc4) + acc1 += 1 + acc2 += 2 + val newUpdates = tm.accumulatorUpdates().map { a => (a.id, a) }.toMap + assert(newUpdates.contains(acc1.id)) + assert(newUpdates.contains(acc2.id)) + assert(newUpdates.contains(acc3.id)) + assert(newUpdates.contains(acc4.id)) + assert(newUpdates(acc1.id).name === "a") + assert(newUpdates(acc2.id).name === "b") + assert(newUpdates(acc3.id).name === "c") + assert(newUpdates(acc4.id).name === "d") + assert(newUpdates(acc1.id).update === Some(1)) + assert(newUpdates(acc2.id).update === Some(2)) + assert(newUpdates(acc3.id).update === Some(0)) + assert(newUpdates(acc4.id).update === Some(0)) + assert(!newUpdates(acc3.id).internal) + assert(!newUpdates(acc3.id).countFailedValues) + assert(newUpdates(acc4.id).internal) + assert(newUpdates(acc4.id).countFailedValues) + assert(newUpdates.values.map(_.update).forall(_.isDefined)) + assert(newUpdates.values.map(_.value).forall(_.isEmpty)) + assert(newUpdates.size === internalAccums.size + 4) + } + + test("existing values in accums") { + // set shuffle read accum before passing it into TaskMetrics + val accums1 = InternalAccumulator.create() + val srAccum = accums1.find(_.name === Some(shuffleRead.FETCH_WAIT_TIME)) + assert(srAccum.isDefined) + srAccum.get.asInstanceOf[Accumulator[Long]] += 10L + val tm1 = new TaskMetrics(accums1) + assert(tm1.shuffleReadMetrics.isDefined) + assert(tm1.shuffleWriteMetrics.isEmpty) + assert(tm1.inputMetrics.isEmpty) + assert(tm1.outputMetrics.isEmpty) + // set shuffle write accum before passing it into TaskMetrics + val accums2 = InternalAccumulator.create() + val swAccum = accums2.find(_.name === Some(shuffleWrite.RECORDS_WRITTEN)) + assert(swAccum.isDefined) + swAccum.get.asInstanceOf[Accumulator[Long]] += 10L + val tm2 = new TaskMetrics(accums2) + assert(tm2.shuffleReadMetrics.isEmpty) + assert(tm2.shuffleWriteMetrics.isDefined) + assert(tm2.inputMetrics.isEmpty) + assert(tm2.outputMetrics.isEmpty) + // set input accum before passing it into TaskMetrics + val accums3 = InternalAccumulator.create() + val inAccum = accums3.find(_.name === Some(input.RECORDS_READ)) + assert(inAccum.isDefined) + inAccum.get.asInstanceOf[Accumulator[Long]] += 10L + val tm3 = new TaskMetrics(accums3) + assert(tm3.shuffleReadMetrics.isEmpty) + assert(tm3.shuffleWriteMetrics.isEmpty) + assert(tm3.inputMetrics.isDefined) + assert(tm3.outputMetrics.isEmpty) + // set output accum before passing it into TaskMetrics + val accums4 = InternalAccumulator.create() + val outAccum = accums4.find(_.name === Some(output.RECORDS_WRITTEN)) + assert(outAccum.isDefined) + outAccum.get.asInstanceOf[Accumulator[Long]] += 10L + val tm4 = new TaskMetrics(accums4) + assert(tm4.shuffleReadMetrics.isEmpty) + assert(tm4.shuffleWriteMetrics.isEmpty) + assert(tm4.inputMetrics.isEmpty) + assert(tm4.outputMetrics.isDefined) + // do not modify any accums before creating TaskMetrics; nothing should be set + val tm5 = new TaskMetrics + assert(tm5.shuffleReadMetrics.isEmpty) + assert(tm5.shuffleWriteMetrics.isEmpty) + assert(tm5.inputMetrics.isEmpty) + assert(tm5.outputMetrics.isEmpty) + } + + test("from accumulator updates") { + val accumUpdates1 = InternalAccumulator.create().map { a => + AccumulableInfo(a.id, a.name.orNull, Some(3L), None, a.isInternal, a.countFailedValues) + } + val metrics1 = TaskMetrics.fromAccumulatorUpdates(0L, accumUpdates1) + assertUpdatesEquals(metrics1.accumulatorUpdates(), accumUpdates1) + // Test this with additional accumulators. Only the ones registered with `Accumulators` + // will show up in the reconstructed TaskMetrics. In practice, all accumulators created + // on the driver, internal or not, should be registered with `Accumulators` at some point. + // Here we show that reconstruction will succeed even if there are unregistered accumulators. + val param = IntAccumulatorParam + val registeredAccums = Seq( + new Accumulator(0, param, Some("a"), internal = true, countFailedValues = true), + new Accumulator(0, param, Some("b"), internal = true, countFailedValues = false), + new Accumulator(0, param, Some("c"), internal = false, countFailedValues = true), + new Accumulator(0, param, Some("d"), internal = false, countFailedValues = false)) + val unregisteredAccums = Seq( + new Accumulator(0, param, Some("e"), internal = true, countFailedValues = true), + new Accumulator(0, param, Some("f"), internal = true, countFailedValues = false)) + registeredAccums.foreach(Accumulators.register) + registeredAccums.foreach { a => assert(Accumulators.originals.contains(a.id)) } + unregisteredAccums.foreach { a => assert(!Accumulators.originals.contains(a.id)) } + // set some values in these accums + registeredAccums.zipWithIndex.foreach { case (a, i) => a.setValue(i) } + unregisteredAccums.zipWithIndex.foreach { case (a, i) => a.setValue(i) } + val registeredAccumInfos = registeredAccums.map(makeInfo) + val unregisteredAccumInfos = unregisteredAccums.map(makeInfo) + val accumUpdates2 = accumUpdates1 ++ registeredAccumInfos ++ unregisteredAccumInfos + val metrics2 = TaskMetrics.fromAccumulatorUpdates(0L, accumUpdates2) + // accumulators that were not registered with `Accumulators` will not show up + assertUpdatesEquals(metrics2.accumulatorUpdates(), accumUpdates1 ++ registeredAccumInfos) } + + + /* --------------------- * + | Helper test methods | + * --------------------- */ + + /** + * Assert that the following three things are equal to `value`: + * (1) TaskMetrics value + * (2) TaskMetrics accumulator update value + * (3) Original accumulator value + */ + private def assertValueEquals( + tm: TaskMetrics, + tmValue: TaskMetrics => Any, + accums: Seq[Accumulator[_]], + metricName: String, + value: Any, + assertEquals: (Any, Any) => Unit = (x: Any, y: Any) => assert(x === y)): Unit = { + assertEquals(tmValue(tm), value) + val accum = accums.find(_.name == Some(metricName)) + assert(accum.isDefined) + assertEquals(accum.get.value, value) + val accumUpdate = tm.accumulatorUpdates().find(_.name == metricName) + assert(accumUpdate.isDefined) + assert(accumUpdate.get.value === None) + assertEquals(accumUpdate.get.update, Some(value)) + } + + /** + * Assert that two lists of accumulator updates are equal. + * Note: this does NOT check accumulator ID equality. + */ + private def assertUpdatesEquals( + updates1: Seq[AccumulableInfo], + updates2: Seq[AccumulableInfo]): Unit = { + assert(updates1.size === updates2.size) + updates1.zip(updates2).foreach { case (info1, info2) => + // do not assert ID equals here + assert(info1.name === info2.name) + assert(info1.update === info2.update) + assert(info1.value === info2.value) + assert(info1.internal === info2.internal) + assert(info1.countFailedValues === info2.countFailedValues) + } + } + + /** + * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the + * info as an accumulator update. + */ + private def makeInfo(a: Accumulable[_, _]): AccumulableInfo = { + new AccumulableInfo( + a.id, a.name.orNull, Some(a.value), None, a.isInternal, a.countFailedValues) + } + } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 1d37eb3c7386..078cffc0cb81 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -126,12 +126,12 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } writer.commitAndClose() val bytesWritten = writeMetrics.bytesWritten - val writeTime = writeMetrics.shuffleWriteTime + val writeTime = writeMetrics.writeTime assert(writeMetrics.recordsWritten === 1000) writer.commitAndClose() assert(writeMetrics.recordsWritten === 1000) assert(writeMetrics.bytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + assert(writeMetrics.writeTime === writeTime) } test("revertPartialWritesAndClose() should be idempotent") { @@ -144,12 +144,12 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } writer.revertPartialWritesAndClose() val bytesWritten = writeMetrics.bytesWritten - val writeTime = writeMetrics.shuffleWriteTime + val writeTime = writeMetrics.writeTime assert(writeMetrics.recordsWritten === 0) writer.revertPartialWritesAndClose() assert(writeMetrics.recordsWritten === 0) assert(writeMetrics.bytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + assert(writeMetrics.writeTime === writeTime) } test("fileSegment() can only be called after commitAndClose() has been called") { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index d84adbbf9b03..0b7483e3b06a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -576,7 +576,7 @@ private[spark] object JsonProtocolSuite extends SparkFunSuite { private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { assert(metrics1.bytesWritten === metrics2.bytesWritten) - assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime) + assert(metrics1.writeTime === metrics2.writeTime) } private def assertEquals(metrics1: InputMetrics, metrics2: InputMetrics) { From 00a12a468e1f83f6fce61b88ce494059e512df21 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Jan 2016 17:24:25 -0800 Subject: [PATCH 64/65] Add fine-grained test for collecting accumulators during failures --- .../org/apache/spark/executor/Executor.scala | 1 - .../spark/executor/TaskMetricsSuite.scala | 91 ++++++++++--------- .../spark/scheduler/DAGSchedulerSuite.scala | 26 +----- .../spark/scheduler/TaskContextSuite.scala | 53 +++++++++++ 4 files changed, 100 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 3f0bc38e8d72..9340b71389de 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -252,7 +252,6 @@ private[spark] class Executor( } // Note: accumulator updates must be collected after TaskMetrics is updated - // TODO: add a test val accumUpdates = task.collectAccumulatorUpdates() val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index aea38c946458..885f424c2340 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -22,10 +22,11 @@ import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel, TestBlockId} -class TaskMetricsSuite extends SparkFunSuite { self => +class TaskMetricsSuite extends SparkFunSuite { import AccumulatorParam._ import InternalAccumulator._ import StorageLevel._ + import TaskMetricsSuite._ test("create") { val internalAccums = InternalAccumulator.create() @@ -101,8 +102,8 @@ class TaskMetricsSuite extends SparkFunSuite { self => import shuffleRead._ val accums = InternalAccumulator.create() val tm = new TaskMetrics(accums) - def assertValueEquals[T](tmValue: ShuffleReadMetrics => T, name: String, value: T): Unit = { - self.assertValueEquals(tm, tm => tmValue(tm.shuffleReadMetrics.get), accums, name, value) + def assertValEquals[T](tmValue: ShuffleReadMetrics => T, name: String, value: T): Unit = { + assertValueEquals(tm, tm => tmValue(tm.shuffleReadMetrics.get), accums, name, value) } // create shuffle read metrics assert(tm.shuffleReadMetrics.isEmpty) @@ -111,12 +112,12 @@ class TaskMetricsSuite extends SparkFunSuite { self => assert(tm.shuffleReadMetrics.isDefined) val sr = tm.shuffleReadMetrics.get // initial values - assertValueEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 0) - assertValueEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 0) - assertValueEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 0L) - assertValueEquals(_.localBytesRead, LOCAL_BYTES_READ, 0L) - assertValueEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 0L) - assertValueEquals(_.recordsRead, RECORDS_READ, 0L) + assertValEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 0) + assertValEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 0) + assertValEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 0L) + assertValEquals(_.localBytesRead, LOCAL_BYTES_READ, 0L) + assertValEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 0L) + assertValEquals(_.recordsRead, RECORDS_READ, 0L) // set and increment values sr.setRemoteBlocksFetched(100) sr.setRemoteBlocksFetched(10) @@ -143,20 +144,20 @@ class TaskMetricsSuite extends SparkFunSuite { self => sr.incRecordsRead(6L) sr.incRecordsRead(6L) // assert new values exist - assertValueEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 12) - assertValueEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 24) - assertValueEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 36L) - assertValueEquals(_.localBytesRead, LOCAL_BYTES_READ, 48L) - assertValueEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 60L) - assertValueEquals(_.recordsRead, RECORDS_READ, 72L) + assertValEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 12) + assertValEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 24) + assertValEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 36L) + assertValEquals(_.localBytesRead, LOCAL_BYTES_READ, 48L) + assertValEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 60L) + assertValEquals(_.recordsRead, RECORDS_READ, 72L) } test("mutating shuffle write metrics values") { import shuffleWrite._ val accums = InternalAccumulator.create() val tm = new TaskMetrics(accums) - def assertValueEquals[T](tmValue: ShuffleWriteMetrics => T, name: String, value: T): Unit = { - self.assertValueEquals(tm, tm => tmValue(tm.shuffleWriteMetrics.get), accums, name, value) + def assertValEquals[T](tmValue: ShuffleWriteMetrics => T, name: String, value: T): Unit = { + assertValueEquals(tm, tm => tmValue(tm.shuffleWriteMetrics.get), accums, name, value) } // create shuffle write metrics assert(tm.shuffleWriteMetrics.isEmpty) @@ -164,9 +165,9 @@ class TaskMetricsSuite extends SparkFunSuite { self => assert(tm.shuffleWriteMetrics.isDefined) val sw = tm.shuffleWriteMetrics.get // initial values - assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 0L) - assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) - assertValueEquals(_.writeTime, WRITE_TIME, 0L) + assertValEquals(_.bytesWritten, BYTES_WRITTEN, 0L) + assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assertValEquals(_.writeTime, WRITE_TIME, 0L) // increment and decrement values sw.incBytesWritten(100L) sw.incBytesWritten(10L) // 100 + 10 @@ -179,17 +180,17 @@ class TaskMetricsSuite extends SparkFunSuite { self => sw.incWriteTime(300L) sw.incWriteTime(30L) // assert new values exist - assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 108L) - assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 216L) - assertValueEquals(_.writeTime, WRITE_TIME, 330L) + assertValEquals(_.bytesWritten, BYTES_WRITTEN, 108L) + assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 216L) + assertValEquals(_.writeTime, WRITE_TIME, 330L) } test("mutating input metrics values") { import input._ val accums = InternalAccumulator.create() val tm = new TaskMetrics(accums) - def assertValueEquals(tmValue: InputMetrics => Any, name: String, value: Any): Unit = { - self.assertValueEquals(tm, tm => tmValue(tm.inputMetrics.get), accums, name, value, + def assertValEquals(tmValue: InputMetrics => Any, name: String, value: Any): Unit = { + assertValueEquals(tm, tm => tmValue(tm.inputMetrics.get), accums, name, value, (x: Any, y: Any) => assert(x.toString === y.toString)) } // create input metrics @@ -198,9 +199,9 @@ class TaskMetricsSuite extends SparkFunSuite { self => assert(tm.inputMetrics.isDefined) val in = tm.inputMetrics.get // initial values - assertValueEquals(_.bytesRead, BYTES_READ, 0L) - assertValueEquals(_.recordsRead, RECORDS_READ, 0L) - assertValueEquals(_.readMethod, READ_METHOD, DataReadMethod.Memory) + assertValEquals(_.bytesRead, BYTES_READ, 0L) + assertValEquals(_.recordsRead, RECORDS_READ, 0L) + assertValEquals(_.readMethod, READ_METHOD, DataReadMethod.Memory) // set and increment values in.setBytesRead(1L) in.setBytesRead(2L) @@ -208,17 +209,17 @@ class TaskMetricsSuite extends SparkFunSuite { self => in.incRecordsRead(2L) in.setReadMethod(DataReadMethod.Disk) // assert new values exist - assertValueEquals(_.bytesRead, BYTES_READ, 2L) - assertValueEquals(_.recordsRead, RECORDS_READ, 3L) - assertValueEquals(_.readMethod, READ_METHOD, DataReadMethod.Disk) + assertValEquals(_.bytesRead, BYTES_READ, 2L) + assertValEquals(_.recordsRead, RECORDS_READ, 3L) + assertValEquals(_.readMethod, READ_METHOD, DataReadMethod.Disk) } test("mutating output metrics values") { import output._ val accums = InternalAccumulator.create() val tm = new TaskMetrics(accums) - def assertValueEquals(tmValue: OutputMetrics => Any, name: String, value: Any): Unit = { - self.assertValueEquals(tm, tm => tmValue(tm.outputMetrics.get), accums, name, value, + def assertValEquals(tmValue: OutputMetrics => Any, name: String, value: Any): Unit = { + assertValueEquals(tm, tm => tmValue(tm.outputMetrics.get), accums, name, value, (x: Any, y: Any) => assert(x.toString === y.toString)) } // create input metrics @@ -227,9 +228,9 @@ class TaskMetricsSuite extends SparkFunSuite { self => assert(tm.outputMetrics.isDefined) val out = tm.outputMetrics.get // initial values - assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 0L) - assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) - assertValueEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) + assertValEquals(_.bytesWritten, BYTES_WRITTEN, 0L) + assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assertValEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) // set values out.setBytesWritten(1L) out.setBytesWritten(2L) @@ -237,11 +238,11 @@ class TaskMetricsSuite extends SparkFunSuite { self => out.setRecordsWritten(4L) out.setWriteMethod(DataWriteMethod.Hadoop) // assert new values exist - assertValueEquals(_.bytesWritten, BYTES_WRITTEN, 2L) - assertValueEquals(_.recordsWritten, RECORDS_WRITTEN, 4L) + assertValEquals(_.bytesWritten, BYTES_WRITTEN, 2L) + assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 4L) // Note: this doesn't actually test anything, but there's only one DataWriteMethod // so we can't set it to anything else - assertValueEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) + assertValEquals(_.writeMethod, WRITE_METHOD, DataWriteMethod.Hadoop) } test("merging multiple shuffle read metrics") { @@ -420,11 +421,11 @@ class TaskMetricsSuite extends SparkFunSuite { self => // accumulators that were not registered with `Accumulators` will not show up assertUpdatesEquals(metrics2.accumulatorUpdates(), accumUpdates1 ++ registeredAccumInfos) } +} - /* --------------------- * - | Helper test methods | - * --------------------- */ +// This extends SparkFunSuite only because we want its `assert` method. +private[spark] object TaskMetricsSuite extends SparkFunSuite { /** * Assert that the following three things are equal to `value`: @@ -432,7 +433,7 @@ class TaskMetricsSuite extends SparkFunSuite { self => * (2) TaskMetrics accumulator update value * (3) Original accumulator value */ - private def assertValueEquals( + def assertValueEquals( tm: TaskMetrics, tmValue: TaskMetrics => Any, accums: Seq[Accumulator[_]], @@ -453,7 +454,7 @@ class TaskMetricsSuite extends SparkFunSuite { self => * Assert that two lists of accumulator updates are equal. * Note: this does NOT check accumulator ID equality. */ - private def assertUpdatesEquals( + def assertUpdatesEquals( updates1: Seq[AccumulableInfo], updates2: Seq[AccumulableInfo]): Unit = { assert(updates1.size === updates2.size) @@ -471,7 +472,7 @@ class TaskMetricsSuite extends SparkFunSuite { self => * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - private def makeInfo(a: Accumulable[_, _]): AccumulableInfo = { + def makeInfo(a: Accumulable[_, _]): AccumulableInfo = { new AccumulableInfo( a.id, a.name.orNull, Some(a.value), None, a.isInternal, a.countFailedValues) } 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 761874d4aeb6..5c3ee56c9b72 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -190,8 +190,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def beforeEach(): Unit = { super.beforeEach() - // This means use 1 core and allow up to 4 failed tasks - sc = new SparkContext("local[1, 4]", "DAGSchedulerSuite") + sc = new SparkContext("local", "DAGSchedulerSuite") sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() sparkListener.failedStages.clear() @@ -1593,29 +1592,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(Accumulators.get(acc3.id).get.value === 18L) } - test("accumulators are updated on exception failures (end-to-end)") { - import AccumulatorParam._ - // Create 2 accumulators, one that counts failed values and another that doesn't - val acc1 = new Accumulator( - 0L, LongAccumulatorParam, Some("ingenieur"), internal = false, countFailedValues = true) - val acc2 = new Accumulator( - 0L, LongAccumulatorParam, Some("boulangere"), internal = false, countFailedValues = false) - // Fail first 3 attempts of every task. This means each task should be run 4 times. - sc.parallelize(1 to 10, 10).map { i => - acc1 += 1 - acc2 += 1 - if (TaskContext.get.attemptNumber() <= 2) { - throw new Exception("you did something wrong") - } else { - 0 - } - }.count() - // The one that counts failed values should be 4x the one that didn't, - // since we ran each task 4 times - assert(Accumulators.get(acc1.id).get.value === 40L) - assert(Accumulators.get(acc2.id).get.value === 10L) - } - test("reduce tasks should be placed locally with map output") { // Create an shuffleMapRdd with 1 partition val shuffleMapRdd = new MyRDD(sc, 1, Nil) 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 599ef1383792..e743c253a368 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -22,7 +22,9 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ +import org.apache.spark.executor.TaskMetricsSuite import org.apache.spark.metrics.source.JvmSource +import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} @@ -97,6 +99,57 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark }.collect() assert(attemptIdsWithFailedTask.toSet === Set(0, 1)) } + + test("accumulators are updated on exception failures") { + // This means use 1 core and 4 max task failures + sc = new SparkContext("local[1,4]", "test") + val param = AccumulatorParam.LongAccumulatorParam + // Create 2 accumulators, one that counts failed values and another that doesn't + val acc1 = new Accumulator(0L, param, Some("x"), internal = false, countFailedValues = true) + val acc2 = new Accumulator(0L, param, Some("y"), internal = false, countFailedValues = false) + // Fail first 3 attempts of every task. This means each task should be run 4 times. + sc.parallelize(1 to 10, 10).map { i => + acc1 += 1 + acc2 += 1 + if (TaskContext.get.attemptNumber() <= 2) { + throw new Exception("you did something wrong") + } else { + 0 + } + }.count() + // The one that counts failed values should be 4x the one that didn't, + // since we ran each task 4 times + assert(Accumulators.get(acc1.id).get.value === 40L) + assert(Accumulators.get(acc2.id).get.value === 10L) + } + + test("failed tasks collect only accumulators whose values count during failures") { + sc = new SparkContext("local", "test") + val param = AccumulatorParam.LongAccumulatorParam + val acc1 = new Accumulator(0L, param, Some("x"), internal = false, countFailedValues = true) + val acc2 = new Accumulator(0L, param, Some("y"), internal = false, countFailedValues = false) + val initialAccums = InternalAccumulator.create() + // Create a dummy task. We won't end up running this; we just want to collect + // accumulator updates from it. + val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]]) { + context = new TaskContextImpl(0, 0, 0L, 0, + new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), + SparkEnv.get.metricsSystem, + initialAccums) + context.taskMetrics.registerAccumulator(acc1) + context.taskMetrics.registerAccumulator(acc2) + override def runTask(tc: TaskContext): Int = 0 + } + // First, simulate task success. This should give us all the accumulators. + val accumUpdates1 = task.collectAccumulatorUpdates(taskFailed = false) + val accumUpdates2 = (initialAccums ++ Seq(acc1, acc2)).map(TaskMetricsSuite.makeInfo) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates1, accumUpdates2) + // Now, simulate task failures. This should give us only the accums that count failed values. + val accumUpdates3 = task.collectAccumulatorUpdates(taskFailed = true) + val accumUpdates4 = (initialAccums ++ Seq(acc1)).map(TaskMetricsSuite.makeInfo) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates3, accumUpdates4) + } + } private object TaskContextSuite { From ed9de9c084b0a8df41b24768404c63cb331c8a25 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Jan 2016 10:01:55 -0800 Subject: [PATCH 65/65] Fix style --- .../scala/org/apache/spark/scheduler/TaskContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e743c253a368..784bcc1e0264 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -23,8 +23,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.TaskMetricsSuite -import org.apache.spark.metrics.source.JvmSource 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.util.{TaskCompletionListener, TaskCompletionListenerException}